You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@buildstream.apache.org by tv...@apache.org on 2021/01/27 07:38:41 UTC

[buildstream] 01/03: Refactor remote asset user configuration

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

tvb pushed a commit to branch tristan/change-remote-config
in repository https://gitbox.apache.org/repos/asf/buildstream.git

commit c6973cc7344404bff209fdac015a5e747a4bd2f6
Author: Tristan van Berkom <tr...@codethink.co.uk>
AuthorDate: Sun Jan 24 13:45:15 2021 +0900

    Refactor remote asset user configuration
    
    This is a full rewrite of how configuration data pertaining to
    AssetCaches, such as the artifact cache and source caches is loaded, which
    before this refactor used to be spread out across several modules.
    
    With the previous refactor of RemoteSpec/RemoteExecutionSpec, we now
    have the configuration loading performed by the appropriate modules
    Context and Project, where the final decision is made by Context, at
    which point the Context will dictate to the AssetCaches which remotes
    to use in which way.
    
    In addition to this refactor, the priorities of remotes are brought more
    in line with BuildStream general policy for user configuration data, so
    instead of compounding remotes found at different levels, we allow user
    configuration to properly override remotes suggested by project configurations,
    the order is now:
    
      * CLI provided `--remote` arguments override anything else, for all projects
        being processed in the given session.
    
      * Remotes defined in buildstream.conf (user configuration) on a per project
        basis specify the full list of remotes which will be used to process that project,
        overriding anything at lower levels.
    
      * Globally defined remotes in the user configuration provide the defaults
        for processing all projects in the session, overriding any suggestions
        made by project defined recommendations.
    
      * Remotes defined in project.conf (project configuration) are treated
        as recommendations, and are overridden by afore mentioned user configuration.
    
    This priority shuffle is inline with the proposal[0]
    
    Another side effect of this refactor is that we no longer support specifying
    a single asset cache for any of the cache types, they must always be
    specified in YAML as lists.
    
    Rationale: It is more clear that it is a list if we force it to be a list.
    This feature could have been useful for convenience, except that it adds no
    convenience at all; when specifying a list with a single element or just
    specifying the dictionary directly, there is no additional yaml (only one space
    becomes a '-' instead of a ' ').
    
    Seeing as it wasn't buying any convenience, it seems pointless to keep
    this around.
    
    Summary of changes:
    
      * _project.py:
    
        Now parse the project recommended remotes using RemoteSpec.new_from_node()
    
      * _context.py:
    
        Now parse the global remotes and project specific overrides using RemoteSpec.new_from_node()
    
        Also Context is now in charge of initializing the asset caches, which
        already technically belong to the Context object, which provides the
        properties which are responsible for lazy instantiation of the caches.
    
        A new Context.initialize_remotes() method is added here for that purpose.
    
      * _stream.py:
    
        Updated to call Context.initialize_remotes() instead of delegating this activity
        to the AssetCaches themselves.
    
        Also renamed the `use_artifact_config` and `use_source_config` arguments used in
        this file to `connect_artifact_cache` and `connect_source_cache`, this simplifies
        the code slightly.
    
      * _assetcache.py:
    
        Any remaining parsing code has been removed from here.
    
        All of the code which used to decide which cache to use in which order
        has been replaced, an asset cache is now given the full list of
        RemoteSpec objects it is expected to communicate with in a given session,
        accompanied by a dictionary of RemoteSpec lists which define which remotes
        to use for each given project it might encounter during the session.
    
        A new method AssetCache.get_remotes() is added for the convenience of
        subclasses so that they might obtain the remotes they need.
    
      * _artifactcache.py, _sourcecache.py, _elementsourcescache.py:
    
        Updated for new API changes, mostly this means using AssetCache.get_remotes().
    
      * tests: Updated test cases to ensure we always specify caches as lists.
    
    [0]: https://lists.apache.org/thread.html/rf2da9830e2fa918357f99a6021e55fc43df876f0b19d43f68802f083%40%3Cdev.buildstream.apache.org%3E
---
 src/buildstream/_artifactcache.py       |  49 ++--
 src/buildstream/_assetcache.py          | 400 +++++++++-----------------------
 src/buildstream/_context.py             | 166 ++++++++++---
 src/buildstream/_elementsourcescache.py |  13 +-
 src/buildstream/_project.py             |  29 +--
 src/buildstream/_sourcecache.py         |  15 +-
 src/buildstream/_stream.py              | 115 +++------
 tests/artifactcache/capabilities.py     |   6 +-
 tests/artifactcache/config.py           |  73 ++----
 tests/artifactcache/junctions.py        |   2 +-
 tests/artifactcache/pull.py             |  15 +-
 tests/artifactcache/push.py             |  14 +-
 tests/frontend/artifact_checkout.py     |   2 +-
 tests/frontend/artifact_delete.py       |   2 +-
 tests/frontend/artifact_pull.py         |   2 +-
 tests/frontend/artifact_show.py         |   2 +-
 tests/frontend/buildcheckout.py         |   6 +-
 tests/frontend/default_target.py        |   2 +-
 tests/frontend/pull.py                  |  40 ++--
 tests/frontend/push.py                  |  40 ++--
 tests/frontend/remote-caches.py         |   8 +-
 tests/frontend/workspace.py             |   2 +-
 tests/integration/artifact.py           |   6 +-
 tests/integration/cachedfail.py         |   4 +-
 tests/integration/pullbuildtrees.py     |  12 +-
 tests/integration/shell.py              |   2 +-
 tests/integration/shellbuildtrees.py    |  10 +-
 tests/remotecache/simple.py             |   2 +-
 tests/remoteexecution/buildtree.py      |   2 +-
 tests/remoteexecution/partial.py        |   5 +-
 tests/sourcecache/capabilities.py       |   8 +-
 tests/sourcecache/fetch.py              |   2 +-
 tests/sourcecache/push.py               |  10 +-
 tests/sourcecache/workspace.py          |   6 +-
 34 files changed, 450 insertions(+), 622 deletions(-)

diff --git a/src/buildstream/_artifactcache.py b/src/buildstream/_artifactcache.py
index d73c7f0..a6bdc99 100644
--- a/src/buildstream/_artifactcache.py
+++ b/src/buildstream/_artifactcache.py
@@ -37,10 +37,6 @@ REMOTE_ASSET_ARTIFACT_URN_TEMPLATE = "urn:fdc:buildstream.build:2020:artifact:{}
 #     context (Context): The BuildStream context
 #
 class ArtifactCache(AssetCache):
-
-    spec_name = "artifact_cache_specs"
-    config_node_name = "artifacts"
-
     def __init__(self, context):
         super().__init__(context)
 
@@ -48,12 +44,6 @@ class ArtifactCache(AssetCache):
         self._basedir = context.artifactdir
         os.makedirs(self._basedir, exist_ok=True)
 
-    def update_mtime(self, ref):
-        try:
-            os.utime(os.path.join(self._basedir, ref))
-        except FileNotFoundError as e:
-            raise ArtifactError("Couldn't find artifact: {}".format(ref)) from e
-
     # preflight():
     #
     # Preflight check.
@@ -88,7 +78,7 @@ class ArtifactCache(AssetCache):
     #     ([str]) - A list of artifact names as generated in LRU order
     #
     def list_artifacts(self, *, glob=None):
-        return [ref for _, ref in sorted(list(self._list_refs_mtimes(self._basedir, glob_expr=glob)))]
+        return [ref for _, ref in sorted(list(self.list_refs_mtimes(self._basedir, glob_expr=glob)))]
 
     # remove():
     #
@@ -101,7 +91,7 @@ class ArtifactCache(AssetCache):
     #
     def remove(self, ref):
         try:
-            self._remove_ref(ref)
+            self.remove_ref(ref)
         except AssetCacheError as e:
             raise ArtifactError("{}".format(e)) from e
 
@@ -123,9 +113,7 @@ class ArtifactCache(AssetCache):
         project = element._get_project()
         display_key = element._get_display_key()
 
-        index_remotes = [r for r in self._index_remotes[project] if r.spec.push]
-        storage_remotes = [r for r in self._storage_remotes[project] if r.spec.push]
-
+        index_remotes, storage_remotes = self.get_remotes(project.name, True)
         artifact_proto = artifact._get_proto()
         artifact_digest = self.cas.add_object(buffer=artifact_proto.SerializeToString())
 
@@ -176,10 +164,12 @@ class ArtifactCache(AssetCache):
         artifact_name = element.get_artifact_name(key=key)
         uri = REMOTE_ASSET_ARTIFACT_URN_TEMPLATE.format(artifact_name)
 
+        index_remotes, storage_remotes = self.get_remotes(project.name, False)
+
         errors = []
         # Start by pulling our artifact proto, so that we know which
         # blobs to pull
-        for remote in self._index_remotes[project]:
+        for remote in index_remotes:
             remote.init()
             try:
                 element.status("Pulling artifact {} <- {}".format(display_key, remote))
@@ -205,7 +195,7 @@ class ArtifactCache(AssetCache):
 
         errors = []
         # If we do, we can pull it!
-        for remote in self._storage_remotes[project]:
+        for remote in storage_remotes:
             remote.init()
             try:
                 element.status("Pulling data for artifact {} <- {}".format(display_key, remote))
@@ -240,7 +230,8 @@ class ArtifactCache(AssetCache):
     #     digest (Digest): The digest of the tree
     #
     def pull_tree(self, project, digest):
-        for remote in self._storage_remotes[project]:
+        _, storage_remotes = self.get_remotes(project.name, False)
+        for remote in storage_remotes:
             digest = self.cas.pull_tree(remote, digest)
 
             if digest:
@@ -261,12 +252,7 @@ class ArtifactCache(AssetCache):
     #     (ArtifactError): if there was an error
     #
     def push_message(self, project, message):
-
-        if self._has_push_remotes:
-            push_remotes = [r for r in self._storage_remotes[project] if r.spec.push]
-        else:
-            push_remotes = []
-
+        _, push_remotes = self.get_remotes(project.name, True)
         if not push_remotes:
             raise ArtifactError(
                 "push_message was called, but no remote artifact " + "servers are configured as push remotes."
@@ -305,7 +291,9 @@ class ArtifactCache(AssetCache):
     #     missing_blobs (list): The Digests of the blobs to fetch
     #
     def fetch_missing_blobs(self, project, missing_blobs):
-        for remote in self._index_remotes[project]:
+
+        index_remotes, _ = self.get_remotes(project.name, False)
+        for remote in index_remotes:
             if not missing_blobs:
                 break
 
@@ -332,8 +320,7 @@ class ArtifactCache(AssetCache):
         if not missing_blobs:
             return []
 
-        push_remotes = [r for r in self._storage_remotes[project] if r.spec.push]
-
+        _, push_remotes = self.get_remotes(project.name, True)
         remote_missing_blobs_list = []
 
         for remote in push_remotes:
@@ -358,13 +345,15 @@ class ArtifactCache(AssetCache):
     #    (bool): True if the element is available remotely
     #
     def check_remotes_for_element(self, element):
+        project = element._get_project()
+        index_remotes, _ = self.get_remotes(project.name, False)
+
         # If there are no remotes
-        if not self._index_remotes:
+        if not index_remotes:
             return False
 
-        project = element._get_project()
         ref = element.get_artifact_name()
-        for remote in self._index_remotes[project]:
+        for remote in index_remotes:
             remote.init()
 
             if self._query_remote(ref, remote):
diff --git a/src/buildstream/_assetcache.py b/src/buildstream/_assetcache.py
index 71c9cb6..66e2182 100644
--- a/src/buildstream/_assetcache.py
+++ b/src/buildstream/_assetcache.py
@@ -18,28 +18,22 @@
 #
 import os
 import re
-from itertools import chain
-from typing import TYPE_CHECKING
+from typing import List, Dict, Tuple, Iterable, Optional
 import grpc
 
 from . import utils
-from . import _yaml
-from ._cas import CASRemote
-from ._exceptions import AssetCacheError, LoadError, RemoteError
+from .node import MappingNode
+from ._cas import CASRemote, CASCache
+from ._exceptions import AssetCacheError, RemoteError
 from ._remotespec import RemoteSpec, RemoteType
 from ._remote import BaseRemote
 from ._protos.build.bazel.remote.asset.v1 import remote_asset_pb2, remote_asset_pb2_grpc
 from ._protos.google.rpc import code_pb2
 
 
-if TYPE_CHECKING:
-    from typing import Optional, Type
-    from ._exceptions import BstError
-
-
 class AssetRemote(BaseRemote):
-    def __init__(self, *args, **kwargs):
-        super().__init__(*args, **kwargs)
+    def __init__(self, spec):
+        super().__init__(spec)
         self.fetch_service = None
         self.push_service = None
 
@@ -257,174 +251,137 @@ class AssetRemote(BaseRemote):
             raise AssetCacheError("PushDirectory failed with status {}: {}".format(e.code().name, e.details())) from e
 
 
-# Base Asset Cache for Caches to derive from
+# RemotePair()
 #
-class AssetCache:
+# A pair of remotes which corresponds to a RemoteSpec, we
+# need separate remote objects for the index and the storage so
+# we store them together for each RemoteSpec here.
+#
+# Either members of the RemotePair may be None, in case that
+# the user specified a diffrerent RemoteSpec for indexing and
+# for storage.
+#
+# Both members may also be None, in the case that we were unable
+# to establish a connection to this remote at initialization time.
+#
+class RemotePair:
+    def __init__(self, cas: CASCache, spec: RemoteSpec):
+        self.index: Optional[AssetRemote] = None
+        self.storage: Optional[CASRemote] = None
+        self.error: Optional[str] = None
 
-    # None of these should ever be called in the base class, but this appeases
-    # pylint to some degree
-    spec_name = None  # type: str
-    config_node_name = None  # type: str
+        try:
+            if spec.remote_type in [RemoteType.INDEX, RemoteType.ALL]:
+                index = AssetRemote(spec)
+                index.check()
+                self.index = index
+            if spec.remote_type in [RemoteType.STORAGE, RemoteType.ALL]:
+                storage = CASRemote(spec, cas)
+                storage.check()
+                self.storage = storage
+        except RemoteError as e:
+            self.error = str(e)
 
+
+# Base Asset Cache for Caches to derive from
+#
+class AssetCache:
     def __init__(self, context):
         self.context = context
-        self.cas = context.get_cascache()
+        self.cas: CASCache = context.get_cascache()
 
-        self._remotes_setup = False  # Check to prevent double-setup of remotes
-        # Per-project list of Remote instances.
-        self._storage_remotes = {}
-        self._index_remotes = {}
+        # Table of RemotePair objects
+        self._remotes: Dict[RemoteSpec, RemotePair] = {}
 
-        self.global_remote_specs = []
-        self.project_remote_specs = {}
+        # Table of prioritized RemoteSpecs which are valid for each project
+        self._project_specs: Dict[str, List[RemoteSpec]] = {}
 
-        self._has_fetch_remotes = False
-        self._has_push_remotes = False
+        self._has_fetch_remotes: bool = False
+        self._has_push_remotes: bool = False
 
         self._basedir = None
 
-    # close_grpc_channels():
-    #
-    # Close open gRPC channels.
-    #
-    def close_grpc_channels(self):
-        # Close all remotes and their gRPC channels
-        for project_remotes in chain(self._index_remotes.values(), self._storage_remotes.values()):
-            for remote in project_remotes:
-                remote.close()
-
     # release_resources():
     #
     # Release resources used by AssetCache.
     #
     def release_resources(self):
-        self.close_grpc_channels()
 
-    # specs_from_config_node()
+        # Close all remotes and their gRPC channels
+        for remote in self._remotes.values():
+            if remote.index:
+                remote.index.close()
+            if remote.storage:
+                remote.storage.close()
+
+    # setup_remotes():
     #
-    # Parses the configuration of remote artifact caches from a config block.
+    # Sets up which remotes to use
     #
     # Args:
-    #   config_node (dict): The config block, which may contain a key defined by cls.config_node_name
-    #   basedir (str): The base directory for relative paths
+    #    specs: The active remote specs
+    #    project_specs: List of specs for each project
     #
-    # Returns:
-    #   A list of RemoteSpec instances.
-    #
-    # Raises:
-    #   LoadError, if the config block contains invalid keys.
-    #
-    @classmethod
-    def specs_from_config_node(cls, config_node, basedir=None):
-        cache_specs = []
+    def setup_remotes(self, specs: Iterable[RemoteSpec], project_specs: Dict[str, List[RemoteSpec]]):
 
-        try:
-            artifacts = [config_node.get_mapping(cls.config_node_name)]
-        except LoadError:
-            try:
-                artifacts = config_node.get_sequence(cls.config_node_name, default=[])
-            except LoadError:
-                provenance = config_node.get_node(cls.config_node_name).get_provenance()
-                raise _yaml.LoadError(
-                    "{}: '{}' must be a single remote mapping, or a list of mappings".format(
-                        provenance, cls.config_node_name
-                    ),
-                    _yaml.LoadErrorReason.INVALID_DATA,
-                )
+        # Hold on to the project specs
+        self._project_specs = project_specs
 
-        for spec_node in artifacts:
-            cache_specs.append(RemoteSpec.new_from_node(spec_node))
+        with self.context.messenger.timed_activity("Initializing remote caches", silent_nested=True):
+            for spec in specs:
+                # This can be called multiple times, ensure that we only try
+                # to instantiate each remote once.
+                #
+                if spec in self._remotes:
+                    continue
 
-        return cache_specs
+                remote = RemotePair(self.cas, spec)
+                if remote.error:
+                    self.context.messenger.warn("Failed to initialize remote {}: {}".format(spec.url, remote.error))
 
-    # setup_remotes():
+                self._remotes[spec] = remote
+
+        # Determine overall existance of push or fetch remotes
+        self._has_fetch_remotes = any(remote.storage for _, remote in self._remotes.items()) and any(
+            remote.index for _, remote in self._remotes.items()
+        )
+        self._has_push_remotes = any(spec.push and remote.storage for spec, remote in self._remotes.items()) and any(
+            spec.push and remote.index for spec, remote in self._remotes.items()
+        )
+
+    # get_remotes():
     #
-    # Sets up which remotes to use
+    # List the index remotes and storage remotes available for fetching
     #
     # Args:
-    #    use_config (bool): Whether to use project configuration
-    #    remote_url (str): Remote cache URL
-    #    reinitialize (bool): Whether to re-initialize the remotes a second time
-    #
-    # This requires that all of the projects which are to be processed in the session
-    # have already been loaded and are observable in the Context.
-    #
-    def setup_remotes(self, *, use_config=False, remote_url=None, reinitialize=False):
-
-        # In some edge cases we need to reinitialize
-        if reinitialize:
-            self._storage_remotes = {}
-            self._index_remotes = {}
-            self._remotes_setup = False
-
-        # Ensure we do not double-initialise since this can be expensive
-        if self._remotes_setup:
-            return
-
-        self._remotes_setup = True
-
-        # Initialize remote caches. We allow the commandline to override
-        # the user config in some cases (for example `bst artifact push --remote=...`).
-        has_remote_caches = False
-        if remote_url:
-            self._set_remotes([RemoteSpec(RemoteType.ALL, remote_url, push=True)])
-            has_remote_caches = True
-        if use_config:
-            for project in self.context.get_projects():
-                caches = self._configured_remote_cache_specs(self.context, project)
-                if caches:  # caches is a list of RemoteSpec instances
-                    self._set_remotes(caches, project=project)
-                    has_remote_caches = True
-
-        def remote_failed(remote, error):
-            self.context.messenger.warn("Failed to initialize remote {}: {}".format(remote.url, error))
-
-        if has_remote_caches:
-            with self.context.messenger.timed_activity("Initializing remote caches", silent_nested=True):
-                self.initialize_remotes(on_failure=remote_failed)
-
-    # Notify remotes that forking is disabled
-    def notify_fork_disabled(self):
-        for project in self._index_remotes:
-            for remote in self._index_remotes[project]:
-                remote.notify_fork_disabled()
-        for project in self._storage_remotes:
-            for remote in self._storage_remotes[project]:
-                remote.notify_fork_disabled()
-
-    # initialize_remotes():
-    #
-    # This will contact each remote cache.
+    #    project_name: The project name
+    #    push: Whether pushing is required for this remote
     #
-    # Args:
-    #     on_failure (callable): Called if we fail to contact one of the caches.
+    # Returns:
+    #    index_remotes: The index remotes
+    #    storage_remotes: The storage remotes
     #
-    def initialize_remotes(self, *, on_failure=None):
-        index_remotes, storage_remotes = self._create_remote_instances(on_failure=on_failure)
-
-        # Assign remote instances to their respective projects
-        for project in self.context.get_projects():
-            # Get the list of specs that should be considered for this
-            # project
-            remote_specs = self.global_remote_specs.copy()
-            if project in self.project_remote_specs:
-                remote_specs.extend(self.project_remote_specs[project])
+    def get_remotes(self, project_name: str, push: bool) -> Tuple[List[AssetRemote], List[CASRemote]]:
+        try:
+            project_specs = self._project_specs[project_name]
+        except KeyError:
+            # Technically this shouldn't happen, but here is a defensive return none the less.
+            return [], []
 
-            # De-duplicate the list
-            remote_specs = list(utils._deduplicate(remote_specs))
+        index_remotes = []
+        storage_remotes = []
+        for spec in project_specs:
 
-            def get_remotes(remote_list, remote_specs):
-                for remote_spec in remote_specs:
-                    # If a remote_spec didn't make it into the remotes
-                    # dict, that means we can't access it, and it has been
-                    # disabled for this session.
-                    if remote_spec not in remote_list:
-                        continue
+            if push and not spec.push:
+                continue
 
-                    yield remote_list[remote_spec]
+            remote = self._remotes[spec]
+            if remote.index:
+                index_remotes.append(remote.index)
+            if remote.storage:
+                storage_remotes.append(remote.storage)
 
-            self._index_remotes[project] = list(get_remotes(index_remotes, remote_specs))
-            self._storage_remotes[project] = list(get_remotes(storage_remotes, remote_specs))
+        return index_remotes, storage_remotes
 
     # has_fetch_remotes():
     #
@@ -443,9 +400,10 @@ class AssetCache:
             # At least one (sub)project has fetch remotes
             return True
         else:
+            project = plugin._get_project()
+            index_remotes, storage_remotes = self.get_remotes(project.name, False)
+
             # Check whether the specified element's project has fetch remotes
-            index_remotes = self._index_remotes[plugin._get_project()]
-            storage_remotes = self._storage_remotes[plugin._get_project()]
             return index_remotes and storage_remotes
 
     # has_push_remotes():
@@ -465,143 +423,13 @@ class AssetCache:
             # At least one (sub)project has push remotes
             return True
         else:
-            # Check whether the specified element's project has push remotes
-            index_remotes = self._index_remotes[plugin._get_project()]
-            storage_remotes = self._storage_remotes[plugin._get_project()]
-            return any(remote.spec.push for remote in index_remotes) and any(
-                remote.spec.push for remote in storage_remotes
-            )
+            project = plugin._get_project()
+            index_remotes, storage_remotes = self.get_remotes(project.name, True)
 
-    ################################################
-    #               Local Private Methods          #
-    ################################################
-
-    # _configured_remote_cache_specs():
-    #
-    # Return the list of configured remotes for a given project, in priority
-    # order. This takes into account the user and project configuration.
-    #
-    # Args:
-    #     context (Context): The BuildStream context
-    #     project (Project): The BuildStream project
-    #
-    # Returns:
-    #   A list of RemoteSpec instances describing the remote caches.
-    #
-    @classmethod
-    def _configured_remote_cache_specs(cls, context, project):
-        project_overrides = context.get_overrides(project.name)
-        project_extra_specs = cls.specs_from_config_node(project_overrides)
-
-        project_specs = getattr(project, cls.spec_name)
-        context_specs = getattr(context, cls.spec_name)
-
-        return list(utils._deduplicate(project_extra_specs + project_specs + context_specs))
-
-    # _create_remote_instances():
-    #
-    # Create the global set of Remote instances, including
-    # project-specific and global instances, ensuring that all of them
-    # are accessible.
-    #
-    # Args:
-    #     on_failure (Callable[[Remote,Exception],None]):
-    #     What do do when a remote doesn't respond.
-    #
-    # Returns:
-    #    (Dict[RemoteSpec, AssetRemote], Dict[RemoteSpec, CASRemote]) -
-    #    The created remote instances, index first, storage last.
-    #
-    def _create_remote_instances(self, *, on_failure=None):
-        # Create a flat list of all remote specs, global or
-        # project-specific
-        remote_specs = self.global_remote_specs.copy()
-        for project in self.project_remote_specs:
-            remote_specs.extend(self.project_remote_specs[project])
-
-        # By de-duplicating it after we flattened the list, we ensure
-        # that we never instantiate the same remote twice. This
-        # de-duplication also preserves their order.
-        remote_specs = list(utils._deduplicate(remote_specs))
-
-        # Now let's create a dict of this, indexed by their specs, so
-        # that we can later assign them to the right projects.
-        index_remotes = {}
-        storage_remotes = {}
-        for remote_spec in remote_specs:
-            try:
-                index, storage = self._instantiate_remote(remote_spec)
-            except RemoteError as err:
-                if on_failure:
-                    on_failure(remote_spec, str(err))
-                    continue
-
-                raise
-
-            # Finally, we can instantiate the remote. Note that
-            # NamedTuples are hashable, so we can use them as pretty
-            # low-overhead keys.
-            if index:
-                index_remotes[remote_spec] = index
-            if storage:
-                storage_remotes[remote_spec] = storage
-
-        self._has_fetch_remotes = storage_remotes and index_remotes
-        self._has_push_remotes = any(spec.push for spec in storage_remotes) and any(
-            spec.push for spec in index_remotes
-        )
-
-        return index_remotes, storage_remotes
-
-    # _instantiate_remote()
-    #
-    # Instantiate a remote given its spec, asserting that it is
-    # reachable - this may produce two remote instances (a storage and
-    # an index remote as specified by the class variables).
-    #
-    # Args:
-    #
-    #    remote_spec (RemoteSpec): The spec of the remote to
-    #                              instantiate.
-    #
-    # Returns:
-    #
-    #    (Tuple[Remote|None, Remote|None]) - The remotes, index remote
-    #    first, storage remote second. One must always be specified,
-    #    the other may be None.
-    #
-    def _instantiate_remote(self, remote_spec):
-        # Our remotes can be index, storage or both. In either case,
-        # we need to use a different type of Remote for our calls, so
-        # we create two objects here
-        index = None
-        storage = None
-        if remote_spec.remote_type in [RemoteType.INDEX, RemoteType.ALL]:
-            index = AssetRemote(remote_spec)  # pylint: disable=not-callable
-            index.check()
-        if remote_spec.remote_type in [RemoteType.STORAGE, RemoteType.ALL]:
-            storage = CASRemote(remote_spec, self.cas)
-            storage.check()
-
-        return (index, storage)
-
-    # _set_remotes():
-    #
-    # Set the list of remote caches. If project is None, the global list of
-    # remote caches will be set, which is used by all projects. If a project is
-    # specified, the per-project list of remote caches will be set.
-    #
-    # Args:
-    #     remote_specs (list): List of ArtifactCacheSpec instances, in priority order.
-    #     project (Project): The Project instance for project-specific remotes
-    def _set_remotes(self, remote_specs, *, project=None):
-        if project is None:
-            # global remotes
-            self.global_remote_specs = remote_specs
-        else:
-            self.project_remote_specs[project] = remote_specs
+            # Check whether the specified element's project has fetch remotes
+            return bool(index_remotes and storage_remotes)
 
-    # _list_refs_mtimes()
+    # list_refs_mtimes()
     #
     # List refs in a directory, given a base path. Also returns the
     # associated mtimes
@@ -613,7 +441,7 @@ class AssetCache:
     # Returns:
     #     (iter (mtime, filename)]): iterator of tuples of mtime and refs
     #
-    def _list_refs_mtimes(self, base_path, *, glob_expr=None):
+    def list_refs_mtimes(self, base_path, *, glob_expr=None):
         path = base_path
         if glob_expr is not None:
             globdir = os.path.dirname(glob_expr)
@@ -635,7 +463,7 @@ class AssetCache:
                     # Obtain the mtime (the time a file was last modified)
                     yield (os.path.getmtime(ref_path), relative_path)
 
-    # _remove_ref()
+    # remove_ref()
     #
     # Removes a ref.
     #
@@ -649,7 +477,7 @@ class AssetCache:
     #    (AssetCacheError): If the ref didnt exist, or a system error
     #                     occurred while removing it
     #
-    def _remove_ref(self, ref):
+    def remove_ref(self, ref):
         try:
             utils._remove_path_with_parents(self._basedir, ref)
         except FileNotFoundError as e:
diff --git a/src/buildstream/_context.py b/src/buildstream/_context.py
index bae97ab..a32950f 100644
--- a/src/buildstream/_context.py
+++ b/src/buildstream/_context.py
@@ -19,6 +19,9 @@
 
 import os
 import shutil
+
+from typing import List, Dict, Set, Optional
+
 from . import utils
 from . import _site
 from . import _yaml
@@ -29,12 +32,12 @@ from ._profile import Topics, PROFILER
 from ._platform import Platform
 from ._artifactcache import ArtifactCache
 from ._elementsourcescache import ElementSourcesCache
-from ._remotespec import RemoteExecutionSpec
+from ._remotespec import RemoteSpec, RemoteExecutionSpec
 from ._sourcecache import SourceCache
 from ._cas import CASCache, CASLogLevel
 from .types import _CacheBuildTrees, _PipelineSelection, _SchedulerErrorAction
 from ._workspaces import Workspaces, WorkspaceProjectCache
-from .node import Node
+from .node import Node, MappingNode
 
 
 # Context()
@@ -83,12 +86,6 @@ class Context:
         # Default root location for workspaces
         self.workspacedir = None
 
-        # specs for source cache remotes
-        self.source_cache_specs = None
-
-        # The locations from which to push and pull prebuilt artifacts
-        self.artifact_cache_specs = None
-
         # The global remote execution configuration
         self.remote_execution_specs = None
 
@@ -161,15 +158,27 @@ class Context:
         # Whether file contents are required for all artifacts in the local cache
         self.require_artifact_files = True
 
-        # Whether elements must be rebuilt when their dependencies have changed
-        self._strict_build_plan = None
+        # Don't shoot the messenger
+        self.messenger = Messenger()
 
         # Make sure the XDG vars are set in the environment before loading anything
         self._init_xdg()
 
-        self.messenger = Messenger()
-
+        #
         # Private variables
+        #
+
+        # Whether elements must be rebuilt when their dependencies have changed
+        self._strict_build_plan = None
+
+        # Lists of globally configured cache specs
+        self._global_artifact_cache_specs: List[RemoteSpec] = []
+        self._global_source_cache_specs: List[RemoteSpec] = []
+
+        # Set of all actively configured remote specs
+        self._active_artifact_cache_specs: Set[RemoteSpec] = set()
+        self._active_source_cache_specs: Set[RemoteSpec] = set()
+
         self._platform = None
         self._artifactcache = None
         self._elementsourcescache = None
@@ -327,11 +336,13 @@ class Context:
                 LoadErrorReason.INVALID_DATA,
             ) from e
 
-        # Load artifact share configuration
-        self.artifact_cache_specs = ArtifactCache.specs_from_config_node(defaults)
+        # Load artifact remote specs
+        caches = defaults.get_sequence("artifacts", default=[], allowed_types=[MappingNode])
+        self._global_artifact_cache_specs = [RemoteSpec.new_from_node(node) for node in caches]
 
-        # Load source cache config
-        self.source_cache_specs = SourceCache.specs_from_config_node(defaults)
+        # Load source cache remote specs
+        caches = defaults.get_sequence("source-caches", default=[], allowed_types=[MappingNode])
+        self._global_source_cache_specs = [RemoteSpec.new_from_node(node) for node in caches]
 
         # Load the global remote execution config including pull-artifact-files setting
         remote_execution = defaults.get_mapping("remote-execution", default=None)
@@ -439,19 +450,6 @@ class Context:
     def add_project(self, project):
         if not self._projects:
             self._workspaces = Workspaces(project, self._workspace_project_cache)
-
-            #
-            # While loading the first, toplevel project, we can adjust some
-            # global settings which can be overridden on a per toplevel project basis.
-            #
-            override_node = self.get_overrides(project.name)
-            if override_node:
-                remote_execution = override_node.get_mapping("remote-execution", default=None)
-                if remote_execution:
-                    self.pull_artifact_files, self.remote_execution_specs = self._load_remote_execution(
-                        remote_execution
-                    )
-
         self._projects.append(project)
 
     # get_projects():
@@ -470,10 +468,116 @@ class Context:
     # invoked with as opposed to a junctioned subproject.
     #
     # Returns:
-    #    (Project): The Project object
+    #    (Project): The toplevel Project object, or None
     #
     def get_toplevel_project(self):
-        return self._projects[0]
+        try:
+            return self._projects[0]
+        except IndexError:
+            return None
+
+    # initialize_remotes()
+    #
+    # This will resolve what remotes each loaded project will interact
+    # with an initialize the underlying asset cache modules.
+    #
+    # Note that this can be called more than once, in the case that
+    # Stream() has loaded additional projects during the load cycle
+    # and some state needs to be recalculated.
+    #
+    # Args:
+    #    connect_artifact_cache: Whether to try to contact remote artifact caches
+    #    connect_source_cache: Whether to try to contact remote source caches
+    #    artifact_remote: An overriding artifact cache remote, or None
+    #    source_remote: An overriding source cache remote, or None
+    #
+    def initialize_remotes(
+        self,
+        connect_artifact_cache: bool,
+        connect_source_cache: bool,
+        artifact_remote: Optional[RemoteSpec],
+        source_remote: Optional[RemoteSpec],
+    ) -> None:
+
+        # Ensure all projects are fully loaded.
+        for project in self._projects:
+            project.ensure_fully_loaded()
+
+        #
+        # If the global remote execution specs have been overridden by the
+        # toplevel project, then adjust them now that we're all loaded.
+        #
+        project = self.get_toplevel_project()
+        if project:
+            override_node = self.get_overrides(project.name)
+            if override_node:
+                remote_execution = override_node.get_mapping("remote-execution", default=None)
+                if remote_execution:
+                    self.pull_artifact_files, self.remote_execution_specs = self._load_remote_execution(
+                        remote_execution
+                    )
+
+        # Collect a table of which specs apply to each project, these
+        # are calculated here and handed over to the asset caches.
+        #
+        project_artifact_cache_specs: Dict[str, List[RemoteSpec]] = {}
+        project_source_cache_specs: Dict[str, List[RemoteSpec]] = {}
+
+        cli_artifact_remotes = [artifact_remote] if artifact_remote else []
+        cli_source_remotes = [source_remote] if source_remote else []
+
+        #
+        # Maintain our list of remote specs for artifact and source caches
+        #
+        for project in self._projects:
+
+            artifact_specs: List[RemoteSpec] = []
+            source_specs: List[RemoteSpec] = []
+
+            override_node = self.get_overrides(project.name)
+
+            # Resolve which remote specs to use, CLI -> Override -> Global -> Project recommendation
+            if connect_artifact_cache:
+                caches = override_node.get_sequence("artifacts", default=[], allowed_types=[MappingNode])
+                override_artifact_specs: List[RemoteSpec] = [RemoteSpec.new_from_node(node) for node in caches]
+                artifact_specs = (
+                    cli_artifact_remotes
+                    or override_artifact_specs
+                    or self._global_artifact_cache_specs
+                    or project.artifact_cache_specs
+                )
+                artifact_specs = list(utils._deduplicate(artifact_specs))
+
+            if connect_source_cache:
+                caches = override_node.get_sequence("source-caches", default=[], allowed_types=[MappingNode])
+                override_source_specs: List[RemoteSpec] = [RemoteSpec.new_from_node(node) for node in caches]
+                source_specs = (
+                    cli_source_remotes
+                    or override_source_specs
+                    or self._global_source_cache_specs
+                    or project.source_cache_specs
+                )
+                source_specs = list(utils._deduplicate(source_specs))
+
+            # Store them for lookups later on
+            project_artifact_cache_specs[project.name] = artifact_specs
+            project_source_cache_specs[project.name] = source_specs
+
+            #
+            # Now that we know which remote specs are going to be used, maintain
+            # our total set of overall active remote specs, this helps the asset cache
+            # modules to maintain a remote connection for the required remotes.
+            #
+            for spec in artifact_specs:
+                self._active_artifact_cache_specs.add(spec)
+            for spec in source_specs:
+                self._active_source_cache_specs.add(spec)
+
+        # Now initialize the underlying asset caches
+        #
+        self.artifactcache.setup_remotes(self._active_artifact_cache_specs, project_artifact_cache_specs)
+        self.elementsourcescache.setup_remotes(self._active_source_cache_specs, project_source_cache_specs)
+        self.sourcecache.setup_remotes(self._active_source_cache_specs, project_source_cache_specs)
 
     # get_workspaces():
     #
diff --git a/src/buildstream/_elementsourcescache.py b/src/buildstream/_elementsourcescache.py
index 873806e..c80ad4b 100644
--- a/src/buildstream/_elementsourcescache.py
+++ b/src/buildstream/_elementsourcescache.py
@@ -32,10 +32,6 @@ REMOTE_ASSET_SOURCE_URN_TEMPLATE = "urn:fdc:buildstream.build:2020:source:{}"
 #    context (Context): The Buildstream context
 #
 class ElementSourcesCache(AssetCache):
-
-    spec_name = "source_cache_specs"
-    config_node_name = "source-caches"
-
     def __init__(self, context):
         super().__init__(context)
 
@@ -86,11 +82,13 @@ class ElementSourcesCache(AssetCache):
 
         uri = REMOTE_ASSET_SOURCE_URN_TEMPLATE.format(ref)
 
+        index_remotes, storage_remotes = self.get_remotes(project.name, False)
+
         source_digest = None
         errors = []
         # Start by pulling our source proto, so that we know which
         # blobs to pull
-        for remote in self._index_remotes[project]:
+        for remote in index_remotes:
             remote.init()
             try:
                 plugin.status("Pulling source {} <- {}".format(display_key, remote))
@@ -114,7 +112,7 @@ class ElementSourcesCache(AssetCache):
             return False
 
         errors = []
-        for remote in self._storage_remotes[project]:
+        for remote in storage_remotes:
             remote.init()
             try:
                 plugin.status("Pulling data for source {} <- {}".format(display_key, remote))
@@ -160,8 +158,7 @@ class ElementSourcesCache(AssetCache):
 
         uri = REMOTE_ASSET_SOURCE_URN_TEMPLATE.format(ref)
 
-        index_remotes = [r for r in self._index_remotes[project] if r.spec.push]
-        storage_remotes = [r for r in self._storage_remotes[project] if r.spec.push]
+        index_remotes, storage_remotes = self.get_remotes(project.name, True)
 
         source_proto = self.load_proto(sources)
         source_digest = self.cas.add_object(buffer=source_proto.SerializeToString())
diff --git a/src/buildstream/_project.py b/src/buildstream/_project.py
index 6dbc120..d24ee07 100644
--- a/src/buildstream/_project.py
+++ b/src/buildstream/_project.py
@@ -33,9 +33,7 @@ from ._profile import Topics, PROFILER
 from ._exceptions import LoadError
 from .exceptions import LoadErrorReason
 from ._options import OptionPool
-from ._artifactcache import ArtifactCache
-from ._sourcecache import SourceCache
-from .node import ScalarNode, SequenceNode, _assert_symbol_name
+from .node import ScalarNode, SequenceNode, MappingNode, ProvenanceInformation, _assert_symbol_name
 from ._pluginfactory import ElementFactory, SourceFactory, load_plugin_origin
 from .types import CoreWarnings
 from ._projectrefs import ProjectRefs, ProjectRefStorage
@@ -47,7 +45,6 @@ from ._remotespec import RemoteSpec
 
 
 if TYPE_CHECKING:
-    from .node import ProvenanceInformation, MappingNode
     from ._context import Context
 
 
@@ -113,7 +110,7 @@ class Project:
         cli_options: Optional[Dict[str, str]] = None,
         default_mirror: Optional[str] = None,
         parent_loader: Optional[Loader] = None,
-        provenance_node: Optional["ProvenanceInformation"] = None,
+        provenance_node: Optional[ProvenanceInformation] = None,
         search_for_project: bool = True,
         load_project: bool = True,
     ):
@@ -135,7 +132,7 @@ class Project:
         self.config: ProjectConfig = ProjectConfig()
         self.first_pass_config: ProjectConfig = ProjectConfig()
 
-        self.base_environment: Union["MappingNode", Dict[str, str]] = {}  # The base set of environment variables
+        self.base_environment: Union[MappingNode, Dict[str, str]] = {}  # The base set of environment variables
         self.base_env_nocache: List[str] = []  # The base nocache mask (list) for the environment
 
         # Remote specs for communicating with remote services
@@ -145,8 +142,8 @@ class Project:
         self.element_factory: Optional[ElementFactory] = None  # ElementFactory for loading elements
         self.source_factory: Optional[SourceFactory] = None  # SourceFactory for loading sources
 
-        self.sandbox: Optional["MappingNode"] = None
-        self.splits: Optional["MappingNode"] = None
+        self.sandbox: Optional[MappingNode] = None
+        self.splits: Optional[MappingNode] = None
 
         #
         # Private members
@@ -864,11 +861,17 @@ class Project:
         # the values from our loaded configuration dictionary.
         #
 
-        # Load artifacts pull/push configuration for this project
-        self.artifact_cache_specs = ArtifactCache.specs_from_config_node(config, self.directory)
-
-        # Load source caches with pull/push config
-        self.source_cache_specs = SourceCache.specs_from_config_node(config, self.directory)
+        # Load artifact remote specs
+        caches = config.get_sequence("artifacts", default=[], allowed_types=[MappingNode])
+        for node in caches:
+            spec = RemoteSpec.new_from_node(node, self.directory)
+            self.artifact_cache_specs.append(spec)
+
+        # Load source cache remote specs
+        caches = config.get_sequence("source-caches", default=[], allowed_types=[MappingNode])
+        for node in caches:
+            spec = RemoteSpec.new_from_node(node, self.directory)
+            self.source_cache_specs.append(spec)
 
         # Load sandbox environment variables
         self.base_environment = config.get_mapping("environment")
diff --git a/src/buildstream/_sourcecache.py b/src/buildstream/_sourcecache.py
index 475a166..2284d0d 100644
--- a/src/buildstream/_sourcecache.py
+++ b/src/buildstream/_sourcecache.py
@@ -36,10 +36,6 @@ REMOTE_ASSET_SOURCE_URN_TEMPLATE = "urn:fdc:buildstream.build:2020:source:{}"
 #    context (Context): The Buildstream context
 #
 class SourceCache(AssetCache):
-
-    spec_name = "source_cache_specs"
-    config_node_name = "source-caches"
-
     def __init__(self, context):
         super().__init__(context)
 
@@ -117,8 +113,7 @@ class SourceCache(AssetCache):
         project = source._get_project()
         display_key = source._get_brief_display_key()
 
-        index_remotes = self._index_remotes[project]
-        storage_remotes = self._storage_remotes[project]
+        index_remotes, storage_remotes = self.get_remotes(project.name, False)
 
         # First fetch the source directory digest so we know what to pull
         source_digest = None
@@ -172,13 +167,7 @@ class SourceCache(AssetCache):
         ref = source._get_source_name()
         project = source._get_project()
 
-        index_remotes = []
-        storage_remotes = []
-
-        # find configured push remotes for this source
-        if self._has_push_remotes:
-            index_remotes = [r for r in self._index_remotes[project] if r.spec.push]
-            storage_remotes = [r for r in self._storage_remotes[project] if r.spec.push]
+        index_remotes, storage_remotes = self.get_remotes(project.name, True)
 
         pushed_storage = False
         pushed_index = False
diff --git a/src/buildstream/_stream.py b/src/buildstream/_stream.py
index fa2421e..1728bd7 100644
--- a/src/buildstream/_stream.py
+++ b/src/buildstream/_stream.py
@@ -47,6 +47,7 @@ from ._scheduler import (
 from .element import Element
 from ._profile import Topics, PROFILER
 from ._project import ProjectRefStorage
+from ._remotespec import RemoteType, RemoteSpec
 from ._state import State
 from .types import _KeyStrength, _PipelineSelection, _Scope
 from .plugin import Plugin
@@ -137,7 +138,7 @@ class Stream:
     #    targets (list of str): Targets to pull
     #    selection (_PipelineSelection): The selection mode for the specified targets
     #    except_targets (list of str): Specified targets to except from fetching
-    #    use_artifact_config (bool): If artifact remote configs should be loaded
+    #    connect_artifact_cache (bool): Whether to try to contact remote artifact caches
     #    load_artifacts (bool): Whether to load artifacts with artifact names
     #
     # Returns:
@@ -148,7 +149,7 @@ class Stream:
         *,
         selection=_PipelineSelection.NONE,
         except_targets=(),
-        use_artifact_config=False,
+        connect_artifact_cache=False,
         load_artifacts=False,
     ):
         with PROFILER.profile(Topics.LOAD_SELECTION, "_".join(t.replace(os.sep, "-") for t in targets)):
@@ -156,7 +157,7 @@ class Stream:
                 targets,
                 selection=selection,
                 except_targets=except_targets,
-                use_artifact_config=use_artifact_config,
+                connect_artifact_cache=connect_artifact_cache,
                 load_artifacts=load_artifacts,
             )
 
@@ -200,7 +201,7 @@ class Stream:
         else:
             selection = _PipelineSelection.BUILD if scope == _Scope.BUILD else _PipelineSelection.RUN
 
-            elements = self.load_selection((element,), selection=selection, use_artifact_config=True)
+            elements = self.load_selection((element,), selection=selection, connect_artifact_cache=True)
 
             # Get element to stage from `targets` list.
             # If scope is BUILD, it will not be in the `elements` list.
@@ -269,17 +270,13 @@ class Stream:
     #
     def build(self, targets, *, selection=_PipelineSelection.PLAN, ignore_junction_targets=False, remote=None):
 
-        use_config = True
-        if remote:
-            use_config = False
-
         elements = self._load(
             targets,
             selection=selection,
             ignore_junction_targets=ignore_junction_targets,
-            use_artifact_config=use_config,
+            connect_artifact_cache=True,
             artifact_remote_url=remote,
-            use_source_config=True,
+            connect_source_cache=True,
             dynamic_plan=True,
         )
 
@@ -330,15 +327,11 @@ class Stream:
     #
     def fetch(self, targets, *, selection=_PipelineSelection.PLAN, except_targets=None, remote=None):
 
-        use_source_config = True
-        if remote:
-            use_source_config = False
-
         elements = self._load(
             targets,
             selection=selection,
             except_targets=except_targets,
-            use_source_config=use_source_config,
+            connect_source_cache=True,
             source_remote_url=remote,
         )
 
@@ -394,16 +387,8 @@ class Stream:
     #
     def source_push(self, targets, *, selection=_PipelineSelection.NONE, remote=None):
 
-        use_source_config = True
-        if remote:
-            use_source_config = False
-
         elements = self._load(
-            targets,
-            selection=selection,
-            use_source_config=use_source_config,
-            source_remote_url=remote,
-            load_artifacts=True,
+            targets, selection=selection, connect_source_cache=True, source_remote_url=remote, load_artifacts=True,
         )
 
         if not self._sourcecache.has_push_remotes():
@@ -433,15 +418,11 @@ class Stream:
     #
     def pull(self, targets, *, selection=_PipelineSelection.NONE, ignore_junction_targets=False, remote=None):
 
-        use_config = True
-        if remote:
-            use_config = False
-
         elements = self._load(
             targets,
             selection=selection,
             ignore_junction_targets=ignore_junction_targets,
-            use_artifact_config=use_config,
+            connect_artifact_cache=True,
             artifact_remote_url=remote,
             load_artifacts=True,
             attempt_artifact_metadata=True,
@@ -475,15 +456,11 @@ class Stream:
     #
     def push(self, targets, *, selection=_PipelineSelection.NONE, ignore_junction_targets=False, remote=None):
 
-        use_config = True
-        if remote:
-            use_config = False
-
         elements = self._load(
             targets,
             selection=selection,
             ignore_junction_targets=ignore_junction_targets,
-            use_artifact_config=use_config,
+            connect_artifact_cache=True,
             artifact_remote_url=remote,
             load_artifacts=True,
         )
@@ -536,7 +513,7 @@ class Stream:
         elements = self._load(
             (target,),
             selection=selection,
-            use_artifact_config=True,
+            connect_artifact_cache=True,
             load_artifacts=True,
             attempt_artifact_metadata=True,
         )
@@ -624,7 +601,7 @@ class Stream:
     def artifact_show(self, targets, *, selection=_PipelineSelection.NONE):
         # Obtain list of Element and/or ArtifactElement objects
         target_objects = self.load_selection(
-            targets, selection=selection, use_artifact_config=True, load_artifacts=True
+            targets, selection=selection, connect_artifact_cache=True, load_artifacts=True
         )
 
         if self._artifacts.has_fetch_remotes():
@@ -1255,38 +1232,6 @@ class Stream:
 
         return elements, except_elements, artifacts
 
-    # _connect_remotes()
-    #
-    # Connect to the source and artifact remotes.
-    #
-    # Args:
-    #     artifact_url: The url of the artifact server to connect to.
-    #     source_url: The url of the source server to connect to.
-    #     use_artifact_config: Whether to use the artifact config.
-    #     use_source_config: Whether to use the source config.
-    #     reinitialize: Whether to reinitialize from scratch
-    #
-    def _connect_remotes(
-        self,
-        artifact_url: str,
-        source_url: str,
-        use_artifact_config: bool,
-        use_source_config: bool,
-        reinitialize: bool = False,
-    ):
-        # ArtifactCache.setup_remotes expects all projects to be fully loaded
-        for project in self._context.get_projects():
-            project.ensure_fully_loaded()
-
-        # Connect to remote caches, this needs to be done before resolving element state
-        self._artifacts.setup_remotes(
-            use_config=use_artifact_config, remote_url=artifact_url, reinitialize=reinitialize
-        )
-        self._elementsourcescache.setup_remotes(
-            use_config=use_source_config, remote_url=source_url, reinitialize=reinitialize
-        )
-        self._sourcecache.setup_remotes(use_config=use_source_config, remote_url=source_url, reinitialize=reinitialize)
-
     # _resolve_cached_remotely()
     #
     # Checks whether the listed elements are currently cached in
@@ -1411,8 +1356,8 @@ class Stream:
     #    selection (_PipelineSelection): The selection mode for the specified targets
     #    except_targets (list of str): Specified targets to except from fetching
     #    ignore_junction_targets (bool): Whether junction targets should be filtered out
-    #    use_artifact_config (bool): Whether to initialize artifacts with the config
-    #    use_source_config (bool): Whether to initialize remote source caches with the config
+    #    connect_artifact_cache (bool): Whether to try to contact remote artifact caches
+    #    connect_source_cache (bool): Whether to try to contact remote source caches
     #    artifact_remote_url (str): A remote url for initializing the artifacts
     #    source_remote_url (str): A remote url for initializing source caches
     #    dynamic_plan (bool): Require artifacts as needed during the build
@@ -1430,8 +1375,8 @@ class Stream:
         selection=_PipelineSelection.NONE,
         except_targets=(),
         ignore_junction_targets=False,
-        use_artifact_config=False,
-        use_source_config=False,
+        connect_artifact_cache=False,
+        connect_source_cache=False,
         artifact_remote_url=None,
         source_remote_url=None,
         dynamic_plan=False,
@@ -1455,8 +1400,20 @@ class Stream:
         # Hold on to the targets
         self.targets = elements
 
+        # FIXME: Instead of converting the URL to a RemoteSpec here, the CLI needs to
+        #        be enhanced to parse a fully qualified RemoteSpec (including certs etc)
+        #        from the command line, the CLI should be feeding the RemoteSpec through
+        #        the Stream API directly.
+        #
+        artifact_remote = None
+        if artifact_remote_url:
+            artifact_remote = RemoteSpec(RemoteType.ALL, artifact_remote_url, push=True)
+        source_remote = None
+        if source_remote_url:
+            source_remote = RemoteSpec(RemoteType.ALL, source_remote_url, push=True)
+
         # Connect to remote caches, this needs to be done before resolving element state
-        self._connect_remotes(artifact_remote_url, source_remote_url, use_artifact_config, use_source_config)
+        self._context.initialize_remotes(connect_artifact_cache, connect_source_cache, artifact_remote, source_remote)
 
         # In some cases we need to have an actualized artifact, with all of
         # it's metadata, such that we can derive attributes about the artifact
@@ -1482,14 +1439,12 @@ class Stream:
                 artifact_targets, [], rewritable=False, valid_artifact_names=True
             )
 
-            # FIXME:
-            #
-            #    Sadly, we need to reinitialize just because we re-instantiated new projects due to
-            #    downloading artifacts - this could be fixed by addressing the awkward structure
-            #    of remotes in the asset caches.
+            # It can be that new remotes have been added by way of loading new
+            # projects referenced by the new artifact elements, so we need to
+            # ensure those remotes are also initialized.
             #
-            self._connect_remotes(
-                artifact_remote_url, source_remote_url, use_artifact_config, use_source_config, reinitialize=True
+            self._context.initialize_remotes(
+                connect_artifact_cache, connect_source_cache, artifact_remote, source_remote
             )
 
         self.targets += artifacts
diff --git a/tests/artifactcache/capabilities.py b/tests/artifactcache/capabilities.py
index c8a49f9..a74ed6c 100644
--- a/tests/artifactcache/capabilities.py
+++ b/tests/artifactcache/capabilities.py
@@ -27,7 +27,7 @@ def test_artifact_cache_with_missing_capabilities_is_skipped(cli, tmpdir, datafi
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "artifacts": {"url": share.repo, "push": True,},
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
@@ -40,8 +40,8 @@ def test_artifact_cache_with_missing_capabilities_is_skipped(cli, tmpdir, datafi
             # Create a local artifact cache handle
             artifactcache = context.artifactcache
 
-            # Manually setup the CAS remote
-            artifactcache.setup_remotes(use_config=True)
+            # Initialize remotes
+            context.initialize_remotes(True, True, None, None)
 
             assert (
                 not artifactcache.has_fetch_remotes()
diff --git a/tests/artifactcache/config.py b/tests/artifactcache/config.py
index 45682ca..3bf853a 100644
--- a/tests/artifactcache/config.py
+++ b/tests/artifactcache/config.py
@@ -1,13 +1,11 @@
 # Pylint doesn't play well with fixtures and dependency injection from pytest
 # pylint: disable=redefined-outer-name
 
-import itertools
 import os
 
 import pytest
 
 from buildstream._remotespec import RemoteSpec, RemoteType
-from buildstream._artifactcache import ArtifactCache
 from buildstream._project import Project
 from buildstream.utils import _deduplicate
 from buildstream import _yaml
@@ -41,28 +39,12 @@ def configure_remote_caches(override_caches, project_caches=None, user_caches=No
         user_caches = []
 
     user_config = {}
-    if len(user_caches) == 1:
-        user_config["artifacts"] = {
-            "url": user_caches[0].url,
-            "push": user_caches[0].push,
-            "type": type_strings[user_caches[0].remote_type],
-        }
-    elif len(user_caches) > 1:
+    if user_caches:
         user_config["artifacts"] = [
             {"url": cache.url, "push": cache.push, "type": type_strings[cache.remote_type]} for cache in user_caches
         ]
 
-    if len(override_caches) == 1:
-        user_config["projects"] = {
-            "test": {
-                "artifacts": {
-                    "url": override_caches[0].url,
-                    "push": override_caches[0].push,
-                    "type": type_strings[override_caches[0].remote_type],
-                }
-            }
-        }
-    elif len(override_caches) > 1:
+    if override_caches:
         user_config["projects"] = {
             "test": {
                 "artifacts": [
@@ -74,25 +56,14 @@ def configure_remote_caches(override_caches, project_caches=None, user_caches=No
 
     project_config = {}
     if project_caches:
-        if len(project_caches) == 1:
-            project_config.update(
-                {
-                    "artifacts": {
-                        "url": project_caches[0].url,
-                        "push": project_caches[0].push,
-                        "type": type_strings[project_caches[0].remote_type],
-                    }
-                }
-            )
-        elif len(project_caches) > 1:
-            project_config.update(
-                {
-                    "artifacts": [
-                        {"url": cache.url, "push": cache.push, "type": type_strings[cache.remote_type]}
-                        for cache in project_caches
-                    ]
-                }
-            )
+        project_config.update(
+            {
+                "artifacts": [
+                    {"url": cache.url, "push": cache.push, "type": type_strings[cache.remote_type]}
+                    for cache in project_caches
+                ]
+            }
+        )
 
     return user_config, project_config
 
@@ -129,11 +100,13 @@ def test_artifact_cache_precedence(tmpdir, override_caches, project_caches, user
         project = Project(str(project_dir), context)
         project.ensure_fully_loaded()
 
-        # Use the helper from the artifactcache module to parse our configuration.
-        parsed_cache_specs = ArtifactCache._configured_remote_cache_specs(context, project)
+        # Check the specs which the artifact cache thinks are configured
+        context.initialize_remotes(True, True, None, None)
+        artifactcache = context.artifactcache
+        parsed_cache_specs = artifactcache._project_specs[project.name]
 
         # Verify that it was correctly read.
-        expected_cache_specs = list(_deduplicate(itertools.chain(override_caches, project_caches, user_caches)))
+        expected_cache_specs = list(_deduplicate(override_caches or user_caches or project_caches))
         assert parsed_cache_specs == expected_cache_specs
 
 
@@ -199,10 +172,12 @@ def test_only_one(cli, datafiles, override_caches, project_caches, user_caches):
 @pytest.mark.parametrize(
     "artifacts_config",
     (
-        {
-            "url": "http://localhost.test",
-            "auth": {"server-cert": "~/server.crt", "client-cert": "~/client.crt", "client-key": "~/client.key",},
-        },
+        [
+            {
+                "url": "http://localhost.test",
+                "auth": {"server-cert": "~/server.crt", "client-cert": "~/client.crt", "client-key": "~/client.key",},
+            }
+        ],
         [
             {
                 "url": "http://localhost.test",
@@ -245,8 +220,10 @@ def test_paths_for_artifact_config_are_expanded(tmpdir, monkeypatch, artifacts_c
         project = Project(str(project_dir), context)
         project.ensure_fully_loaded()
 
-        # Use the helper from the artifactcache module to parse our configuration.
-        parsed_cache_specs = ArtifactCache._configured_remote_cache_specs(context, project)
+        # Check the specs which the artifact cache thinks are configured
+        context.initialize_remotes(True, True, None, None)
+        artifactcache = context.artifactcache
+        parsed_cache_specs = artifactcache._project_specs[project.name]
 
     if isinstance(artifacts_config, dict):
         artifacts_config = [artifacts_config]
diff --git a/tests/artifactcache/junctions.py b/tests/artifactcache/junctions.py
index e1b7dbf..c180de4 100644
--- a/tests/artifactcache/junctions.py
+++ b/tests/artifactcache/junctions.py
@@ -17,7 +17,7 @@ DATA_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "junctions"
 def project_set_artifacts(project, url):
     project_conf_file = os.path.join(project, "project.conf")
     project_config = _yaml.load(project_conf_file, shortname=None)
-    project_config["artifacts"] = {"url": url, "push": True}
+    project_config["artifacts"] = [{"url": url, "push": True}]
     _yaml.roundtrip_dump(project_config.strip_node_info(), file=project_conf_file)
 
 
diff --git a/tests/artifactcache/pull.py b/tests/artifactcache/pull.py
index 63e6d98..502c350 100644
--- a/tests/artifactcache/pull.py
+++ b/tests/artifactcache/pull.py
@@ -41,7 +41,7 @@ def test_pull(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "artifacts": {"url": share.repo, "push": True,},
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
 
@@ -86,8 +86,8 @@ def test_pull(cli, tmpdir, datafiles):
             # Create a local artifact cache handle
             artifactcache = context.artifactcache
 
-            # Manually setup the CAS remote
-            artifactcache.setup_remotes(use_config=True)
+            # Initialize remotes
+            context.initialize_remotes(True, True, None, None)
 
             assert artifactcache.has_push_remotes(plugin=element), "No remote configured for element target.bst"
             assert artifactcache.pull(element, element_key), "Pull operation failed"
@@ -106,7 +106,7 @@ def test_pull_tree(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "artifacts": {"url": share.repo, "push": True,},
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": rootcache_dir,
         }
 
@@ -138,9 +138,10 @@ def test_pull_tree(cli, tmpdir, datafiles):
             # Retrieve the Directory object from the cached artifact
             artifact_digest = cli.artifact.get_digest(rootcache_dir, element, element_key)
 
+            # Initialize remotes
+            context.initialize_remotes(True, True, None, None)
+
             artifactcache = context.artifactcache
-            # Manually setup the CAS remote
-            artifactcache.setup_remotes(use_config=True)
             assert artifactcache.has_push_remotes()
 
             directory = remote_execution_pb2.Directory()
@@ -161,7 +162,7 @@ def test_pull_tree(cli, tmpdir, datafiles):
             cli.remove_artifact_from_cache(project_dir, "target.bst")
 
             # Assert that we are not cached locally anymore
-            artifactcache.close_grpc_channels()
+            artifactcache.release_resources()
             cas._casd_channel.request_shutdown()
             cas.close_grpc_channels()
             assert cli.get_element_state(project_dir, "target.bst") != "cached"
diff --git a/tests/artifactcache/push.py b/tests/artifactcache/push.py
index 74062ce..48985df 100644
--- a/tests/artifactcache/push.py
+++ b/tests/artifactcache/push.py
@@ -39,9 +39,8 @@ def _push(cli, cache_dir, project_dir, config_file, target):
         for e in element._dependencies(_Scope.ALL):
             e._initialize_state()
 
-        # Manually setup the CAS remotes
-        artifactcache.setup_remotes(use_config=True)
-        artifactcache.initialize_remotes()
+        # Initialize remotes
+        context.initialize_remotes(True, True, None, None)
 
         assert artifactcache.has_push_remotes(plugin=element), "No remote configured for element target.bst"
         assert element._push(), "Push operation failed"
@@ -67,7 +66,7 @@ def test_push(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "artifacts": {"url": share.repo, "push": True,},
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": rootcache_dir,
         }
 
@@ -124,7 +123,7 @@ def test_push_message(tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "artifacts": {"url": share.repo, "push": True,},
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": rootcache_dir,
         }
 
@@ -139,9 +138,8 @@ def test_push_message(tmpdir, datafiles):
             # Create a local artifact cache handle
             artifactcache = context.artifactcache
 
-            # Manually setup the artifact remote
-            artifactcache.setup_remotes(use_config=True)
-            artifactcache.initialize_remotes()
+            # Initialize remotes
+            context.initialize_remotes(True, True, None, None)
             assert artifactcache.has_push_remotes()
 
             command = remote_execution_pb2.Command(
diff --git a/tests/frontend/artifact_checkout.py b/tests/frontend/artifact_checkout.py
index 1375bc3..329d2df 100644
--- a/tests/frontend/artifact_checkout.py
+++ b/tests/frontend/artifact_checkout.py
@@ -39,7 +39,7 @@ def test_checkout(cli, tmpdir, datafiles, deps, expect_exist, expect_noexist, wi
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
         # Build the element to push it to cache
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         # Build it
         result = cli.run(project=project, args=["build", "target-import.bst"])
diff --git a/tests/frontend/artifact_delete.py b/tests/frontend/artifact_delete.py
index 37b9731..90d448b 100644
--- a/tests/frontend/artifact_delete.py
+++ b/tests/frontend/artifact_delete.py
@@ -152,7 +152,7 @@ def test_artifact_delete_pulled_artifact_without_buildtree(cli, tmpdir, datafile
     local_cache = os.path.join(str(tmpdir), "artifacts")
     with create_artifact_share(os.path.join(str(tmpdir), "remote")) as remote:
         cli.configure(
-            {"artifacts": {"url": remote.repo, "push": True}, "cachedir": local_cache,}
+            {"artifacts": [{"url": remote.repo, "push": True}], "cachedir": local_cache,}
         )
 
         # Build the element
diff --git a/tests/frontend/artifact_pull.py b/tests/frontend/artifact_pull.py
index 4fa6b19..926f96e 100644
--- a/tests/frontend/artifact_pull.py
+++ b/tests/frontend/artifact_pull.py
@@ -41,7 +41,7 @@ def test_pull(cli, tmpdir, datafiles, deps, expect_cached, with_project):
         # Build the element to push it to cache, and explicitly configure local cache so we can check it
         local_cache = os.path.join(str(tmpdir), "cache")
         cli.configure(
-            {"cachedir": local_cache, "artifacts": {"url": share.repo, "push": True},}
+            {"cachedir": local_cache, "artifacts": [{"url": share.repo, "push": True}],}
         )
 
         # Build it
diff --git a/tests/frontend/artifact_show.py b/tests/frontend/artifact_show.py
index 2a7131c..7e8f8ee 100644
--- a/tests/frontend/artifact_show.py
+++ b/tests/frontend/artifact_show.py
@@ -184,7 +184,7 @@ def test_artifact_show_element_available_remotely(cli, tmpdir, datafiles):
     local_cache = os.path.join(str(tmpdir), "artifacts")
     with create_artifact_share(os.path.join(str(tmpdir), "remote")) as remote:
         cli.configure(
-            {"artifacts": {"url": remote.repo, "push": True}, "cachedir": local_cache,}
+            {"artifacts": [{"url": remote.repo, "push": True}], "cachedir": local_cache,}
         )
 
         # Build the element
diff --git a/tests/frontend/buildcheckout.py b/tests/frontend/buildcheckout.py
index 6d11906..3a1d650 100644
--- a/tests/frontend/buildcheckout.py
+++ b/tests/frontend/buildcheckout.py
@@ -124,7 +124,7 @@ def test_non_strict_pull_build_strict_checkout(datafiles, cli, tmpdir):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo}})
+        cli.configure({"artifacts": [{"url": share.repo}]})
 
         # First build it in non-strict mode with an artifact server configured.
         # With this configuration BuildStream will attempt to pull the build-only
@@ -1089,7 +1089,7 @@ def test_partial_artifact_checkout_fetch(cli, datafiles, tmpdir):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         result = cli.run(project=project, args=["source", "track", input_name])
         result.assert_success()
@@ -1124,7 +1124,7 @@ def test_partial_checkout_fail(tmpdir, datafiles, cli):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         res = cli.run(project=project, args=["artifact", "checkout", "--pull", build_elt, "--directory", checkout_dir])
         res.assert_main_error(ErrorDomain.STREAM, "uncached-checkout-attempt")
diff --git a/tests/frontend/default_target.py b/tests/frontend/default_target.py
index 60578bb..f6573c6 100644
--- a/tests/frontend/default_target.py
+++ b/tests/frontend/default_target.py
@@ -177,7 +177,7 @@ def test_default_target_push_pull(cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
         # Push the artifacts
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["artifact", "push"])
         result.assert_success()
 
diff --git a/tests/frontend/pull.py b/tests/frontend/pull.py
index f873ad9..6733584 100644
--- a/tests/frontend/pull.py
+++ b/tests/frontend/pull.py
@@ -46,7 +46,7 @@ def test_push_pull_deps(cli, tmpdir, datafiles, deps, expected_states):
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
         # First build the target element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", target])
         result.assert_success()
 
@@ -137,9 +137,7 @@ def test_push_pull_specific_remote(cli, tmpdir, datafiles):
 
         # Configure the default push location to be bad_share; we will assert that
         # nothing actually gets pushed there.
-        cli.configure(
-            {"artifacts": {"url": bad_share.repo, "push": True},}
-        )
+        cli.configure({"artifacts": [{"url": bad_share.repo, "push": True},]})
 
         # Now try `bst artifact push` to the good_share.
         result = cli.run(project=project, args=["artifact", "push", "target.bst", "--remote", good_share.repo])
@@ -175,7 +173,7 @@ def test_push_pull_non_strict(cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
         # First build the target element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}, "projects": {"test": {"strict": False}}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}], "projects": {"test": {"strict": False}}})
         result = cli.run(project=project, args=["build", "target.bst"])
         result.assert_success()
         assert cli.get_element_state(project, "target.bst") == "cached"
@@ -225,7 +223,7 @@ def test_push_pull_cross_junction(cli, tmpdir, datafiles):
         generate_junction(tmpdir, subproject_path, junction_path, store_ref=True)
 
         # First build the target element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", "junction.bst:import-etc.bst"])
         result.assert_success()
         assert cli.get_element_state(project, "junction.bst:import-etc.bst") == "cached"
@@ -290,7 +288,7 @@ def test_pull_missing_blob(cli, tmpdir, datafiles):
     project = str(datafiles)
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         _test_pull_missing_blob(cli, project, share, share)
 
@@ -345,7 +343,7 @@ def test_pull_missing_local_blob(cli, tmpdir, datafiles):
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
         # First build the import-bin element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         result = cli.run(project=project, args=["source", "track", input_name])
         result.assert_success()
@@ -376,7 +374,7 @@ def test_pull_missing_notifies_user(caplog, cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo}})
+        cli.configure({"artifacts": [{"url": share.repo}]})
         result = cli.run(project=project, args=["build", "target.bst"])
 
         result.assert_success()
@@ -393,18 +391,14 @@ def test_build_remote_option(caplog, cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare1")) as shareuser, create_artifact_share(
         os.path.join(str(tmpdir), "artifactshare2")
-    ) as shareproject, create_artifact_share(os.path.join(str(tmpdir), "artifactshare3")) as sharecli:
-
-        # Add shareproject repo url to project.conf
-        with open(os.path.join(project, "project.conf"), "a") as projconf:
-            projconf.write("artifacts:\n  url: {}\n  push: True".format(shareproject.repo))
+    ) as sharecli:
 
         # Configure shareuser remote in user conf
-        cli.configure({"artifacts": {"url": shareuser.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": shareuser.repo, "push": True}]})
 
-        # Push the artifacts to the shareuser and shareproject remotes.
-        # Assert that shareuser and shareproject have the artfifacts cached,
-        # but sharecli doesn't, then delete locally cached elements
+        # Push the artifacts to the shareuser remote.
+        # Assert that shareuser has the artfifacts cached, but sharecli doesn't,
+        # then delete locally cached elements
         result = cli.run(project=project, args=["build", "target.bst"])
         result.assert_success()
         all_elements = ["target.bst", "import-bin.bst", "compose-all.bst"]
@@ -412,11 +406,10 @@ def test_build_remote_option(caplog, cli, tmpdir, datafiles):
             assert element_name in result.get_pushed_elements()
             assert_not_shared(cli, sharecli, project, element_name)
             assert_shared(cli, shareuser, project, element_name)
-            assert_shared(cli, shareproject, project, element_name)
             cli.remove_artifact_from_cache(project, element_name)
 
         # Now check that a build with cli set as sharecli results in nothing being pulled,
-        # as it doesn't have them cached and shareuser/shareproject should be ignored. This
+        # as it doesn't have them cached and shareuser should be ignored. This
         # will however result in the artifacts being built and pushed to it
         result = cli.run(project=project, args=["build", "--remote", sharecli.repo, "target.bst"])
         result.assert_success()
@@ -432,7 +425,6 @@ def test_build_remote_option(caplog, cli, tmpdir, datafiles):
         for element_name in all_elements:
             assert cli.get_element_state(project, element_name) == "cached"
             assert element_name in result.get_pulled_elements()
-        assert shareproject.repo not in result.stderr
         assert shareuser.repo not in result.stderr
         assert sharecli.repo in result.stderr
 
@@ -457,7 +449,7 @@ def test_pull_access_rights(cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", "compose-all.bst"])
         result.assert_success()
 
@@ -519,7 +511,7 @@ def test_pull_artifact(cli, tmpdir, datafiles):
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
         # First build the target element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         result = cli.run(project=project, args=["build", element])
         result.assert_success()
@@ -557,7 +549,7 @@ def test_dynamic_build_plan(cli, tmpdir, datafiles):
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
         # First build the target element and push to the remote.
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", target])
         result.assert_success()
 
diff --git a/tests/frontend/push.py b/tests/frontend/push.py
index 4e39c22..8f71c8a 100644
--- a/tests/frontend/push.py
+++ b/tests/frontend/push.py
@@ -71,9 +71,7 @@ def test_push(cli, tmpdir, datafiles):
 
             # Configure bst to pull but not push from a cache and run `bst artifact push`.
             # This should also fail.
-            cli.configure(
-                {"artifacts": {"url": share1.repo, "push": False},}
-            )
+            cli.configure({"artifacts": [{"url": share1.repo, "push": False},]})
             result = cli.run(project=project, args=["artifact", "push", "target.bst"])
             result.assert_main_error(ErrorDomain.STREAM, None)
 
@@ -127,7 +125,7 @@ def test_push_artifact(cli, tmpdir, datafiles):
                 #        only, but it should probably be fixed.
                 #
                 "scheduler": {"pushers": 1},
-                "artifacts": {"url": share.repo, "push": True,},
+                "artifacts": [{"url": share.repo, "push": True,}],
             }
         )
 
@@ -163,7 +161,7 @@ def test_push_artifact_glob(cli, tmpdir, datafiles):
         assert os.path.exists(os.path.join(local_cache, "artifacts", "refs", artifact_ref))
 
         # Configure artifact share
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         # Run bst artifact push with a wildcard, there is only one artifact
         # matching "test/target/*", even though it can be accessed both by it's
@@ -282,7 +280,7 @@ def test_push_deps(cli, tmpdir, datafiles, deps, expected_states):
                 #        only, but it should probably be fixed.
                 #
                 "scheduler": {"pushers": 1},
-                "artifacts": {"url": share.repo, "push": True,},
+                "artifacts": [{"url": share.repo, "push": True,}],
             }
         )
 
@@ -331,7 +329,7 @@ def test_push_artifacts_all_deps_fails(cli, tmpdir, datafiles):
                 #        only, but it should probably be fixed.
                 #
                 "scheduler": {"pushers": 1},
-                "artifacts": {"url": share.repo, "push": True,},
+                "artifacts": [{"url": share.repo, "push": True,}],
             }
         )
 
@@ -354,9 +352,7 @@ def test_push_after_pull(cli, tmpdir, datafiles):
 
         # Set the scene: share1 has the artifact, share2 does not.
         #
-        cli.configure(
-            {"artifacts": {"url": share1.repo, "push": True},}
-        )
+        cli.configure({"artifacts": [{"url": share1.repo, "push": True},]})
 
         result = cli.run(project=project, args=["build", "target.bst"])
         result.assert_success()
@@ -400,9 +396,7 @@ def test_artifact_expires(cli, datafiles, tmpdir):
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare"), quota=int(22e6)) as share:
 
         # Configure bst to push to the cache
-        cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
-        )
+        cli.configure({"artifacts": [{"url": share.repo, "push": True},]})
 
         # Create and build an element of 15 MB
         create_element_size("element1.bst", project, element_path, [], int(15e6))
@@ -453,7 +447,7 @@ def test_artifact_too_large(cli, datafiles, tmpdir):
 
         # Configure bst to push to the remote cache
         cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
+            {"artifacts": [{"url": share.repo, "push": True}],}
         )
 
         # Create and push a 3MB element
@@ -506,7 +500,7 @@ def test_recently_pulled_artifact_does_not_expire(cli, datafiles, tmpdir):
 
         # Configure bst to push to the cache
         cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
+            {"artifacts": [{"url": share.repo, "push": True}],}
         )
 
         # Create and build 2 elements, one 5 MB and one 15 MB.
@@ -570,7 +564,7 @@ def test_push_cross_junction(cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
         cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
+            {"artifacts": [{"url": share.repo, "push": True}],}
         )
         cli.run(project=project, args=["artifact", "push", "junction.bst:import-etc.bst"])
 
@@ -585,7 +579,7 @@ def test_push_already_cached(caplog, cli, tmpdir, datafiles):
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
 
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", "target.bst"])
 
         result.assert_success()
@@ -610,10 +604,10 @@ def test_build_remote_option(caplog, cli, tmpdir, datafiles):
 
         # Add shareproject repo url to project.conf
         with open(os.path.join(project, "project.conf"), "a") as projconf:
-            projconf.write("artifacts:\n  url: {}\n  push: True".format(shareproject.repo))
+            projconf.write("artifacts:\n- url: {}\n  push: True".format(shareproject.repo))
 
         # Configure shareuser remote in user conf
-        cli.configure({"artifacts": {"url": shareuser.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": shareuser.repo, "push": True}]})
 
         result = cli.run(project=project, args=["build", "--remote", sharecli.repo, "target.bst"])
 
@@ -640,7 +634,7 @@ def test_push_no_strict(caplog, cli, tmpdir, datafiles, buildtrees):
     caplog.set_level(1)
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}, "projects": {"test": {"strict": False}}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}], "projects": {"test": {"strict": False}}})
 
         # First get us a build
         result = cli.run(project=project, args=["build", "target.bst"])
@@ -699,7 +693,7 @@ def test_push_after_rebuild(cli, tmpdir, datafiles):
     assert cli.get_element_state(project, "random.bst") != "cached"
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         # Now rebuild the element and push it
         result = cli.run(project=project, args=["build", "random.bst"])
@@ -724,7 +718,7 @@ def test_push_update_after_rebuild(cli, tmpdir, datafiles):
     )
 
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         # Build the element and push the artifact
         result = cli.run(project=project, args=["build", "random.bst"])
@@ -743,6 +737,6 @@ def test_push_update_after_rebuild(cli, tmpdir, datafiles):
         assert cli.get_element_state(project, "random.bst") == "cached"
 
         # Push the new build
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["artifact", "push", "random.bst"])
         assert result.get_pushed_elements() == ["random.bst"]
diff --git a/tests/frontend/remote-caches.py b/tests/frontend/remote-caches.py
index ebafddf..03d728d 100644
--- a/tests/frontend/remote-caches.py
+++ b/tests/frontend/remote-caches.py
@@ -45,8 +45,8 @@ def test_source_artifact_caches(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
-            "artifacts": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": cachedir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
@@ -80,8 +80,8 @@ def test_source_cache_empty_artifact_cache(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
-            "artifacts": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
+            "artifacts": [{"url": share.repo, "push": True,}],
             "cachedir": cachedir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
diff --git a/tests/frontend/workspace.py b/tests/frontend/workspace.py
index 813e681..3a8bc22 100644
--- a/tests/frontend/workspace.py
+++ b/tests/frontend/workspace.py
@@ -1030,7 +1030,7 @@ def test_external_push_pull(cli, datafiles, tmpdir_factory, guess_element):
         result = cli.run(project=project, args=["-C", workspace, "build", element_name])
         result.assert_success()
 
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
 
         result = cli.run(project=project, args=["-C", workspace, "artifact", "push", *arg_elm])
         result.assert_success()
diff --git a/tests/integration/artifact.py b/tests/integration/artifact.py
index f7d62a5..faefa8f 100644
--- a/tests/integration/artifact.py
+++ b/tests/integration/artifact.py
@@ -55,7 +55,7 @@ def test_cache_buildtrees(cli, tmpdir, datafiles):
     with create_artifact_share(os.path.join(str(tmpdir), "share1")) as share1, create_artifact_share(
         os.path.join(str(tmpdir), "share2")
     ) as share2, create_artifact_share(os.path.join(str(tmpdir), "share3")) as share3:
-        cli.configure({"artifacts": {"url": share1.repo, "push": True}, "cachedir": str(tmpdir)})
+        cli.configure({"artifacts": [{"url": share1.repo, "push": True}], "cachedir": str(tmpdir)})
 
         # Build autotools element with the default behavior of caching buildtrees
         # only when necessary. The artifact should be successfully pushed to the share1 remote
@@ -98,7 +98,7 @@ def test_cache_buildtrees(cli, tmpdir, datafiles):
 
         # Repeat building the artifacts, this time with cache-buildtrees set to
         # 'always' via the cli, as such the buildtree dir should not be empty
-        cli.configure({"artifacts": {"url": share2.repo, "push": True}, "cachedir": str(tmpdir)})
+        cli.configure({"artifacts": [{"url": share2.repo, "push": True}], "cachedir": str(tmpdir)})
         result = cli.run(project=project, args=["--cache-buildtrees", "always", "build", element_name])
         assert result.exit_code == 0
         assert cli.get_element_state(project, element_name) == "cached"
@@ -127,7 +127,7 @@ def test_cache_buildtrees(cli, tmpdir, datafiles):
         # a build
         cli.configure(
             {
-                "artifacts": {"url": share3.repo, "push": True},
+                "artifacts": [{"url": share3.repo, "push": True}],
                 "cachedir": str(tmpdir),
                 "cache": {"cache-buildtrees": "always"},
             }
diff --git a/tests/integration/cachedfail.py b/tests/integration/cachedfail.py
index f72c315..3b59199 100644
--- a/tests/integration/cachedfail.py
+++ b/tests/integration/cachedfail.py
@@ -126,7 +126,7 @@ def test_push_cached_fail(cli, tmpdir, datafiles, on_error):
 
     with create_artifact_share(os.path.join(str(tmpdir), "remote")) as share:
         cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
+            {"artifacts": [{"url": share.repo, "push": True}],}
         )
 
         # Build the element, continuing to finish active jobs on error.
@@ -169,7 +169,7 @@ def test_push_failed_missing_shell(cli, tmpdir, datafiles, on_error):
 
     with create_artifact_share(os.path.join(str(tmpdir), "remote")) as share:
         cli.configure(
-            {"artifacts": {"url": share.repo, "push": True},}
+            {"artifacts": [{"url": share.repo, "push": True}],}
         )
 
         # Build the element, continuing to finish active jobs on error.
diff --git a/tests/integration/pullbuildtrees.py b/tests/integration/pullbuildtrees.py
index 5923ce5..eeeb7a5 100644
--- a/tests/integration/pullbuildtrees.py
+++ b/tests/integration/pullbuildtrees.py
@@ -26,7 +26,7 @@ def default_state(cli, tmpdir, share):
     shutil.rmtree(os.path.join(str(tmpdir), "cas"))
     cli.configure(
         {
-            "artifacts": {"url": share.repo, "push": False},
+            "artifacts": [{"url": share.repo, "push": False}],
             "cachedir": str(tmpdir),
             "cache": {"pull-buildtrees": False},
         }
@@ -50,7 +50,7 @@ def test_pullbuildtrees(cli2, tmpdir, datafiles):
     ) as share2, create_artifact_share(os.path.join(str(tmpdir), "share3")) as share3:
         cli2.configure(
             {
-                "artifacts": {"url": share1.repo, "push": True},
+                "artifacts": [{"url": share1.repo, "push": True}],
                 "cachedir": str(tmpdir),
                 "cache": {"cache-buildtrees": "always"},
             }
@@ -117,7 +117,7 @@ def test_pullbuildtrees(cli2, tmpdir, datafiles):
         # to share2
         result = cli2.run(project=project, args=["artifact", "pull", element_name])
         assert element_name in result.get_pulled_elements()
-        cli2.configure({"artifacts": {"url": share2.repo, "push": True}})
+        cli2.configure({"artifacts": [{"url": share2.repo, "push": True}]})
         result = cli2.run(project=project, args=["artifact", "push", element_name])
         assert element_name not in result.get_pushed_elements()
         assert not share2.get_artifact(cli2.get_artifact_name(project, "test", element_name))
@@ -125,10 +125,10 @@ def test_pullbuildtrees(cli2, tmpdir, datafiles):
         # Assert that after pulling the missing buildtree the element artifact can be
         # successfully pushed to the remote. This will attempt to pull the buildtree
         # from share1 and then a 'complete' push to share2
-        cli2.configure({"artifacts": {"url": share1.repo, "push": False}})
+        cli2.configure({"artifacts": [{"url": share1.repo, "push": False}]})
         result = cli2.run(project=project, args=["--pull-buildtrees", "artifact", "pull", element_name])
         assert element_name in result.get_pulled_elements()
-        cli2.configure({"artifacts": {"url": share2.repo, "push": True}})
+        cli2.configure({"artifacts": [{"url": share2.repo, "push": True}]})
         result = cli2.run(project=project, args=["artifact", "push", element_name])
         assert element_name in result.get_pushed_elements()
         assert share2.get_artifact(cli2.get_artifact_name(project, "test", element_name))
@@ -140,7 +140,7 @@ def test_pullbuildtrees(cli2, tmpdir, datafiles):
         # artifact cannot be pushed.
         result = cli2.run(project=project, args=["artifact", "pull", element_name])
         assert element_name in result.get_pulled_elements()
-        cli2.configure({"artifacts": {"url": share3.repo, "push": True}})
+        cli2.configure({"artifacts": [{"url": share3.repo, "push": True}]})
         result = cli2.run(project=project, args=["--pull-buildtrees", "artifact", "push", element_name])
         assert element_name not in result.get_pulled_elements()
         with cli2.artifact.extract_buildtree(cwd, cwd, artifact_name) as buildtreedir:
diff --git a/tests/integration/shell.py b/tests/integration/shell.py
index a022d86..308b398 100644
--- a/tests/integration/shell.py
+++ b/tests/integration/shell.py
@@ -359,7 +359,7 @@ def test_integration_partial_artifact(cli, datafiles, tmpdir, integration_cache)
 
     # push to an artifact server so we can pull from it later.
     with create_artifact_share(os.path.join(str(tmpdir), "artifactshare")) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}]})
         result = cli.run(project=project, args=["build", element_name])
         result.assert_success()
 
diff --git a/tests/integration/shellbuildtrees.py b/tests/integration/shellbuildtrees.py
index 47ca9f6..cd38839 100644
--- a/tests/integration/shellbuildtrees.py
+++ b/tests/integration/shellbuildtrees.py
@@ -130,7 +130,7 @@ def create_built_artifact_share(tmpdir, cache_buildtrees, integration_cache):
 
     # Create a Cli instance to build and populate the share
     cli = Cli(os.path.join(tmpdir, "cache"))
-    cli.configure({"artifacts": {"url": share.repo, "push": True}, "sourcedir": integration_cache.sources})
+    cli.configure({"artifacts": [{"url": share.repo, "push": True}], "sourcedir": integration_cache.sources})
 
     # Optionally cache build trees
     args = []
@@ -240,7 +240,7 @@ def test_shell_use_cached_buildtree(share_with_buildtrees, datafiles, cli, pull_
     project = str(datafiles)
     element_name = "build-shell/buildtree.bst"
 
-    cli.configure({"artifacts": {"url": share_with_buildtrees.repo}})
+    cli.configure({"artifacts": [{"url": share_with_buildtrees.repo}]})
 
     # Optionally pull the buildtree along with `bst artifact pull`
     maybe_pull_deps(cli, project, element_name, pull_deps, pull_buildtree)
@@ -280,7 +280,7 @@ def test_shell_pull_cached_buildtree(share_with_buildtrees, datafiles, cli, pull
     project = str(datafiles)
     element_name = "build-shell/buildtree.bst"
 
-    cli.configure({"artifacts": {"url": share_with_buildtrees.repo}})
+    cli.configure({"artifacts": [{"url": share_with_buildtrees.repo}]})
 
     # Optionally pull the buildtree along with `bst artifact pull`
     maybe_pull_deps(cli, project, element_name, pull_deps, pull_buildtree)
@@ -319,7 +319,7 @@ def test_shell_use_uncached_buildtree(share_without_buildtrees, datafiles, cli):
     project = str(datafiles)
     element_name = "build-shell/buildtree.bst"
 
-    cli.configure({"artifacts": {"url": share_without_buildtrees.repo}})
+    cli.configure({"artifacts": [{"url": share_without_buildtrees.repo}]})
 
     # Pull everything we would need
     maybe_pull_deps(cli, project, element_name, "all", True)
@@ -342,7 +342,7 @@ def test_shell_pull_uncached_buildtree(share_without_buildtrees, datafiles, cli)
     project = str(datafiles)
     element_name = "build-shell/buildtree.bst"
 
-    cli.configure({"artifacts": {"url": share_without_buildtrees.repo}})
+    cli.configure({"artifacts": [{"url": share_without_buildtrees.repo}]})
 
     # Run the shell and request that required artifacts and buildtrees should be pulled
     result = cli.run(
diff --git a/tests/remotecache/simple.py b/tests/remotecache/simple.py
index 74c44ae..4f8896a 100644
--- a/tests/remotecache/simple.py
+++ b/tests/remotecache/simple.py
@@ -57,7 +57,7 @@ def test_remote_autotools_build_no_cache(cli, datafiles):
     checkout = os.path.join(cli.directory, "checkout")
     element_name = "autotools/amhello.bst"
 
-    cli.configure({"artifacts": {"url": "http://fake.url.service", "push": True}})
+    cli.configure({"artifacts": [{"url": "http://fake.url.service", "push": True}]})
     result = cli.run(project=project, args=["build", element_name])
     result.assert_success()
 
diff --git a/tests/remoteexecution/buildtree.py b/tests/remoteexecution/buildtree.py
index 317747f..4586286 100644
--- a/tests/remoteexecution/buildtree.py
+++ b/tests/remoteexecution/buildtree.py
@@ -41,7 +41,7 @@ def test_buildtree_remote(cli, tmpdir, datafiles):
     assert set(services) == set(["action-cache", "execution", "storage"])
 
     with create_artifact_share(share_path) as share:
-        cli.configure({"artifacts": {"url": share.repo, "push": True}, "cache": {"pull-buildtrees": False}})
+        cli.configure({"artifacts": [{"url": share.repo, "push": True}], "cache": {"pull-buildtrees": False}})
 
         res = cli.run(project=project, args=["--cache-buildtrees", "always", "build", element_name])
         res.assert_success()
diff --git a/tests/remoteexecution/partial.py b/tests/remoteexecution/partial.py
index ec5fabe..7688df4 100644
--- a/tests/remoteexecution/partial.py
+++ b/tests/remoteexecution/partial.py
@@ -70,10 +70,7 @@ def test_build_partial_push(cli, tmpdir, datafiles):
         services = cli.ensure_services()
         assert set(services) == set(["action-cache", "execution", "storage"])
 
-        cli.config["artifacts"] = {
-            "url": share.repo,
-            "push": True,
-        }
+        cli.config["artifacts"] = [{"url": share.repo, "push": True,}]
 
         res = cli.run(project=project, args=["build", element_name])
         res.assert_success()
diff --git a/tests/sourcecache/capabilities.py b/tests/sourcecache/capabilities.py
index 9d41eba..67be749 100644
--- a/tests/sourcecache/capabilities.py
+++ b/tests/sourcecache/capabilities.py
@@ -25,7 +25,7 @@ def test_artifact_cache_with_missing_capabilities_is_skipped(cli, tmpdir, datafi
         # Configure artifact share
         cache_dir = os.path.join(str(tmpdir), "cache")
         user_config_file = str(tmpdir.join("buildstream.conf"))
-        user_config = {"scheduler": {"pushers": 1}, "source-caches": {"url": share.repo,}, "cachedir": cache_dir}
+        user_config = {"scheduler": {"pushers": 1}, "source-caches": [{"url": share.repo,}], "cachedir": cache_dir}
         _yaml.roundtrip_dump(user_config, file=user_config_file)
 
         with dummy_context(config=user_config_file) as context:
@@ -33,12 +33,12 @@ def test_artifact_cache_with_missing_capabilities_is_skipped(cli, tmpdir, datafi
             project = Project(project_dir, context)
             project.ensure_fully_loaded()
 
+            # Initialize remotes
+            context.initialize_remotes(True, True, None, None)
+
             # Create a local artifact cache handle
             sourcecache = context.sourcecache
 
-            # Manually setup the CAS remote
-            sourcecache.setup_remotes(use_config=True)
-
             assert (
                 not sourcecache.has_fetch_remotes()
             ), "System didn't realize the source cache didn't support BuildStream"
diff --git a/tests/sourcecache/fetch.py b/tests/sourcecache/fetch.py
index 76f5508..7b2c63a 100644
--- a/tests/sourcecache/fetch.py
+++ b/tests/sourcecache/fetch.py
@@ -51,7 +51,7 @@ def context_with_source_cache(cli, cache, share, tmpdir):
     user_config_file = str(tmpdir.join("buildstream.conf"))
     user_config = {
         "scheduler": {"pushers": 1},
-        "source-caches": {"url": share.repo,},
+        "source-caches": [{"url": share.repo,}],
         "cachedir": cache,
     }
     _yaml.roundtrip_dump(user_config, file=user_config_file)
diff --git a/tests/sourcecache/push.py b/tests/sourcecache/push.py
index 25a4309..bc591ad 100644
--- a/tests/sourcecache/push.py
+++ b/tests/sourcecache/push.py
@@ -115,7 +115,7 @@ def test_source_push(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
@@ -165,7 +165,7 @@ def test_push_pull(cli, datafiles, tmpdir):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
@@ -203,7 +203,7 @@ def test_push_fail(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
@@ -234,7 +234,7 @@ def test_source_push_build_fail(cli, tmpdir, datafiles):
     with create_artifact_share(os.path.join(str(tmpdir), "share")) as share:
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         cli.configure(user_config)
@@ -275,7 +275,7 @@ def test_push_missing_source_after_build(cli, tmpdir, datafiles):
         user_config_file = str(tmpdir.join("buildstream.conf"))
         user_config = {
             "scheduler": {"pushers": 1},
-            "source-caches": {"url": share.repo, "push": True,},
+            "source-caches": [{"url": share.repo, "push": True,}],
             "cachedir": cache_dir,
         }
         _yaml.roundtrip_dump(user_config, file=user_config_file)
diff --git a/tests/sourcecache/workspace.py b/tests/sourcecache/workspace.py
index bb1ea50..7dccbe0 100644
--- a/tests/sourcecache/workspace.py
+++ b/tests/sourcecache/workspace.py
@@ -72,7 +72,11 @@ def test_workspace_open_no_source_push(tmpdir, datafiles, cli):
 
     with create_artifact_share(share_dir) as share:
         cli.configure(
-            {"cachedir": cache_dir, "scheduler": {"pushers": 1}, "source-caches": {"url": share.repo, "push": True,},}
+            {
+                "cachedir": cache_dir,
+                "scheduler": {"pushers": 1},
+                "source-caches": [{"url": share.repo, "push": True,}],
+            }
         )
 
         # Fetch as in previous test and check it pushes the source