You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@heron.apache.org by ni...@apache.org on 2020/07/14 17:02:46 UTC

[incubator-heron] branch master updated: Updates to Kubernetes scheduler to fix dockerenv issue (#3550)

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

nicknezis pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-heron.git


The following commit(s) were added to refs/heads/master by this push:
     new e1dda9a  Updates to Kubernetes scheduler to fix dockerenv issue (#3550)
e1dda9a is described below

commit e1dda9acfded22ac1de86d73981118488d7a3b19
Author: Nicholas Nezis <ni...@gmail.com>
AuthorDate: Tue Jul 14 13:02:36 2020 -0400

    Updates to Kubernetes scheduler to fix dockerenv issue (#3550)
    
    * Updates to Kubernetes scheduler to create Headless service
    * Use FQDN instead of hostname when running in Kubernetes environment
---
 WORKSPACE                                          |    5 +-
 examples/src/java/BUILD                            |    2 +-
 heron/executor/src/python/heron_executor.py        |    7 +-
 .../scheduler/kubernetes/KubernetesCompat.java     |   79 -
 .../scheduler/kubernetes/KubernetesScheduler.java  |    2 +-
 .../{AppsV1Controller.java => V1Controller.java}   |   88 +-
 heron/stmgr/src/cpp/manager/ckptmgr-client.cpp     |    4 +-
 .../cpp/server/stateful-restorer_unittest.cpp      |    4 +-
 maven_install.json                                 | 5512 ++++++++++----------
 9 files changed, 2856 insertions(+), 2847 deletions(-)

diff --git a/WORKSPACE b/WORKSPACE
index 862c40d..2e35479 100644
--- a/WORKSPACE
+++ b/WORKSPACE
@@ -54,7 +54,7 @@ jetty_version = "9.4.6.v20170531"
 
 jersey_version = "2.25.1"
 
-kubernetes_client_version = "7.0.0"
+kubernetes_client_version = "8.0.0"
 
 load("@rules_jvm_external//:defs.bzl", "maven_install")
 load("@rules_jvm_external//:specs.bzl", "maven")
@@ -73,7 +73,7 @@ maven_install(
         "org.apache.mesos:mesos:0.22.0",
         "com.hashicorp.nomad:nomad-sdk:0.7.0",
         "org.apache.hadoop:hadoop-core:0.20.2",
-        "org.apache.pulsar:pulsar-client:1.19.0-incubating",
+        "org.apache.pulsar:pulsar-client:jar:shaded:1.19.0-incubating",
         "org.apache.kafka:kafka-clients:2.2.0",
         "com.google.apis:google-api-services-storage:v1-rev108-" + google_client_version,
         "org.apache.reef:reef-runtime-yarn:" + reef_version,
@@ -81,7 +81,6 @@ maven_install(
         "org.apache.httpcomponents:httpclient:" + http_client_version,
         "org.apache.httpcomponents:httpmime:" + http_client_version,
         "com.google.apis:google-api-services-storage:v1-rev108-1.22.0",
-        "io.kubernetes:client-java:7.0.0",
         "com.microsoft.dhalion:dhalion:0.2.3",
         "org.objenesis:objenesis:2.1",
         "org.ow2.asm:asm-all:5.1",
diff --git a/examples/src/java/BUILD b/examples/src/java/BUILD
index dc76ece..8d9e823 100644
--- a/examples/src/java/BUILD
+++ b/examples/src/java/BUILD
@@ -29,7 +29,7 @@ java_binary(
         "//heron/common/src/java:basics-java",
         "//heron/simulator/src/java:simulator-java",
         "//third_party/java:kryo",
-        "@maven//:org_apache_pulsar_pulsar_client",
+        "@maven//:org_apache_pulsar_pulsar_client_shaded",
     ],
 )
 
diff --git a/heron/executor/src/python/heron_executor.py b/heron/executor/src/python/heron_executor.py
index cb063aa..68c094d 100755
--- a/heron/executor/src/python/heron_executor.py
+++ b/heron/executor/src/python/heron_executor.py
@@ -140,6 +140,9 @@ def log_pid_for_process(process_name, pid):
 def is_docker_environment():
   return os.path.isfile('/.dockerenv')
 
+def is_kubernetes_environment():
+  return 'POD_NAME' in os.environ
+
 def stdout_log_fn(cmd):
   """Simple function callback that is used to log the streaming output of a subprocess command
   :param cmd: the name of the command which will be added to the log line
@@ -235,7 +238,9 @@ class HeronExecutor:
     # Needed for Docker environments since the hostname of a docker container is the container's
     # id within docker, rather than the host's hostname. NOTE: this 'HOST' env variable is not
     # guaranteed to be set in all Docker executor environments (outside of Marathon)
-    if is_docker_environment():
+    if is_kubernetes_environment():
+      self.master_host = socket.getfqdn()
+    elif is_docker_environment():
       self.master_host = os.environ.get('HOST') if 'HOST' in os.environ else socket.gethostname()
     else:
       self.master_host = socket.gethostname()
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesCompat.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesCompat.java
deleted file mode 100644
index 9ce5fba..0000000
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesCompat.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.heron.scheduler.kubernetes;
-
-import java.io.IOException;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.heron.scheduler.TopologyRuntimeManagementException;
-
-import io.kubernetes.client.openapi.ApiClient;
-import io.kubernetes.client.openapi.ApiException;
-import io.kubernetes.client.openapi.Configuration;
-import io.kubernetes.client.openapi.apis.CoreV1Api;
-
-import okhttp3.Response;
-
-public class KubernetesCompat {
-
-  private static final Logger LOG = Logger.getLogger(KubernetesCompat.class.getName());
-
-  boolean killTopology(String kubernetesUri, String topology, String namespace) {
-    CoreV1Api coreClient;
-    try {
-      final ApiClient apiClient = io.kubernetes.client.util.Config.defaultClient();
-      Configuration.setDefaultApiClient(apiClient);
-      coreClient = new CoreV1Api(apiClient);
-    } catch (IOException e) {
-      LOG.log(Level.SEVERE, "Failed to setup Kubernetes client" + e);
-      throw new RuntimeException(e);
-    }
-
-    // old version deployed topologies as naked pods
-    try {
-      final String labelSelector = KubernetesConstants.LABEL_TOPOLOGY + "=" + topology;
-      final Response response =
-          coreClient.deleteCollectionNamespacedPodCall(namespace, null, null, null, null, null,
-          null, labelSelector, null, null,
-          KubernetesConstants.DELETE_OPTIONS_PROPAGATION_POLICY,
-          null, null, null, null, null).execute();
-
-      if (response.isSuccessful()) {
-        LOG.log(Level.INFO, "Pods for the Job [" + topology
-            + "] in namespace [" + namespace + "] are deleted.");
-        return true;
-      } else {
-        LOG.log(Level.SEVERE, "Error when deleting the Pods of the job ["
-            + topology + "]: in namespace [" + namespace + "]");
-        LOG.log(Level.SEVERE, "Error killing topology message: " + response.message());
-        KubernetesUtils.logResponseBodyIfPresent(LOG, response);
-        throw new TopologyRuntimeManagementException(
-            KubernetesUtils.errorMessageFromResponse(response));
-      }
-    } catch (IOException | ApiException e) {
-      LOG.log(Level.SEVERE, "Error killing topology " + e.getMessage());
-      if (e instanceof ApiException) {
-        LOG.log(Level.SEVERE, "Error details:\n" +  ((ApiException) e).getResponseBody());
-      }
-      return false;
-    }
-  }
-}
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
index 71545be..c35c87d 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
@@ -50,7 +50,7 @@ public class KubernetesScheduler implements IScheduler, IScalable {
   private UpdateTopologyManager updateTopologyManager;
 
   protected KubernetesController getController() {
-    return new AppsV1Controller(configuration, runtimeConfiguration);
+    return new V1Controller(configuration, runtimeConfiguration);
   }
 
   @Override
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/AppsV1Controller.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java
similarity index 86%
rename from heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/AppsV1Controller.java
rename to heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java
index 8724034..619562f 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/AppsV1Controller.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java
@@ -48,6 +48,7 @@ import io.kubernetes.client.openapi.ApiClient;
 import io.kubernetes.client.openapi.ApiException;
 import io.kubernetes.client.openapi.Configuration;
 import io.kubernetes.client.openapi.apis.AppsV1Api;
+import io.kubernetes.client.openapi.apis.CoreV1Api;
 import io.kubernetes.client.openapi.models.V1Container;
 import io.kubernetes.client.openapi.models.V1ContainerPort;
 import io.kubernetes.client.openapi.models.V1EnvVar;
@@ -58,6 +59,8 @@ import io.kubernetes.client.openapi.models.V1ObjectMeta;
 import io.kubernetes.client.openapi.models.V1PodSpec;
 import io.kubernetes.client.openapi.models.V1PodTemplateSpec;
 import io.kubernetes.client.openapi.models.V1ResourceRequirements;
+import io.kubernetes.client.openapi.models.V1Service;
+import io.kubernetes.client.openapi.models.V1ServiceSpec;
 import io.kubernetes.client.openapi.models.V1StatefulSet;
 import io.kubernetes.client.openapi.models.V1StatefulSetSpec;
 import io.kubernetes.client.openapi.models.V1Toleration;
@@ -66,21 +69,23 @@ import io.kubernetes.client.openapi.models.V1VolumeMount;
 
 import okhttp3.Response;
 
-public class AppsV1Controller extends KubernetesController {
+public class V1Controller extends KubernetesController {
 
   private static final Logger LOG =
-      Logger.getLogger(AppsV1Controller.class.getName());
+      Logger.getLogger(V1Controller.class.getName());
 
   private static final String ENV_SHARD_ID = "SHARD_ID";
 
   private final AppsV1Api appsClient;
+  private final CoreV1Api coreClient;
 
-  AppsV1Controller(Config configuration, Config runtimeConfiguration) {
+  V1Controller(Config configuration, Config runtimeConfiguration) {
     super(configuration, runtimeConfiguration);
     try {
       final ApiClient apiClient = io.kubernetes.client.util.Config.defaultClient();
       Configuration.setDefaultApiClient(apiClient);
       appsClient = new AppsV1Api(apiClient);
+      coreClient = new CoreV1Api(apiClient);
     } catch (IOException e) {
       LOG.log(Level.SEVERE, "Failed to setup Kubernetes client" + e);
       throw new RuntimeException(e);
@@ -96,6 +101,16 @@ public class AppsV1Controller extends KubernetesController {
 
     final Resource containerResource = getContainerResource(packingPlan);
 
+    final V1Service topologyService = createTopologyyService();
+    try {
+      final V1Service response =
+          coreClient.createNamespacedService(getNamespace(), topologyService, null,
+              null, null);
+    } catch (ApiException e) {
+      KubernetesUtils.logExceptionWithDetails(LOG, "Error creating topology service", e);
+      throw new TopologySubmissionException(e.getMessage());
+    }
+
     // find the max number of instances in a container so we can open
     // enough ports if remote debugging is enabled.
     int numberOfInstances = 0;
@@ -118,11 +133,9 @@ public class AppsV1Controller extends KubernetesController {
 
   @Override
   boolean killTopology() {
-    return
-        isStatefulSet()
-        ? deleteStatefulSet()
-        :
-        new KubernetesCompat().killTopology(getKubernetesUri(), getTopologyName(), getNamespace());
+    deleteStatefulSet();
+    deleteService();
+    return true;
   }
 
   @Override
@@ -199,6 +212,31 @@ public class AppsV1Controller extends KubernetesController {
         null, null, null);
   }
 
+  boolean deleteService() {
+    try {
+      final Response response = coreClient.deleteNamespacedServiceCall(getTopologyName(),
+          getNamespace(), null, null, 0, null,
+          KubernetesConstants.DELETE_OPTIONS_PROPAGATION_POLICY, null, null).execute();
+
+      if (response.isSuccessful()) {
+        LOG.log(Level.INFO, "Headless Service for the Job [" + getTopologyName()
+            + "] in namespace [" + getNamespace() + "] is deleted.");
+        return true;
+      } else {
+        LOG.log(Level.SEVERE, "Error when deleting the Service of the job ["
+            + getTopologyName() + "] in namespace [" + getNamespace() + "]");
+        LOG.log(Level.SEVERE, "Error killing topoogy message:" + response.message());
+        KubernetesUtils.logResponseBodyIfPresent(LOG, response);
+
+        throw new TopologyRuntimeManagementException(
+            KubernetesUtils.errorMessageFromResponse(response));
+      }
+    } catch (IOException | ApiException e) {
+      KubernetesUtils.logExceptionWithDetails(LOG, "Error deleting topology service", e);
+      return false;
+    }
+  }
+
   boolean deleteStatefulSet() {
     try {
       final Response response = appsClient.deleteNamespacedStatefulSetCall(getTopologyName(),
@@ -211,7 +249,7 @@ public class AppsV1Controller extends KubernetesController {
         return true;
       } else {
         LOG.log(Level.SEVERE, "Error when deleting the StatefulSet of the job ["
-            + getTopologyName() + "]: in namespace [" + getNamespace() + "]");
+            + getTopologyName() + "] in namespace [" + getNamespace() + "]");
         LOG.log(Level.SEVERE, "Error killing topology message: " + response.message());
         KubernetesUtils.logResponseBodyIfPresent(LOG, response);
 
@@ -224,18 +262,6 @@ public class AppsV1Controller extends KubernetesController {
     }
   }
 
-  boolean isStatefulSet() {
-    try {
-      final V1StatefulSet response =
-          appsClient.readNamespacedStatefulSet(getTopologyName(), getNamespace(),
-              null, null, null);
-      return response.getKind().equals("StatefulSet");
-    } catch (ApiException e) {
-      LOG.warning("isStatefulSet check " +  e.getMessage());
-    }
-    return false;
-  }
-
   protected List<String> getExecutorCommand(String containerId) {
     final Map<ExecutorPort, String> ports =
         KubernetesConstants.EXECUTOR_PORTS.entrySet()
@@ -262,6 +288,26 @@ public class AppsV1Controller extends KubernetesController {
     return String.format("%s=${POD_NAME##*-} && echo shardId=${%s}", ENV_SHARD_ID, ENV_SHARD_ID);
   }
 
+  private V1Service createTopologyyService() {
+    final String topologyName = getTopologyName();
+    final Config runtimeConfiguration = getRuntimeConfiguration();
+
+    final V1Service service = new V1Service();
+
+    // setup service metadata
+    final V1ObjectMeta objectMeta = new V1ObjectMeta();
+    objectMeta.name(topologyName);
+    service.setMetadata(objectMeta);
+
+    // create the headless service
+    final V1ServiceSpec serviceSpec = new V1ServiceSpec();
+    serviceSpec.clusterIP("None");
+    serviceSpec.setSelector(getMatchLabels(topologyName));
+
+    service.setSpec(serviceSpec);
+
+    return service;
+  }
 
   private V1StatefulSet createStatefulSet(Resource containerResource, int numberOfInstances) {
     final String topologyName = getTopologyName();
diff --git a/heron/stmgr/src/cpp/manager/ckptmgr-client.cpp b/heron/stmgr/src/cpp/manager/ckptmgr-client.cpp
index 332bc88..61c3e7b 100644
--- a/heron/stmgr/src/cpp/manager/ckptmgr-client.cpp
+++ b/heron/stmgr/src/cpp/manager/ckptmgr-client.cpp
@@ -45,10 +45,10 @@ CkptMgrClient::CkptMgrClient(std::shared_ptr<EventLoop> eventloop, const Network
       ckptmgr_id_(_ckptmgr_id),
       stmgr_id_(_stmgr_id),
       quit_(false),
-      pplan_(nullptr),
       ckpt_saved_watcher_(_ckpt_saved_watcher),
       ckpt_get_watcher_(_ckpt_get_watcher),
-      register_watcher_(_register_watcher) {
+      register_watcher_(_register_watcher),
+      pplan_(nullptr) {
 
   // TODO(nlu): take the value from config
   reconnect_cpktmgr_interval_sec_ = 10;
diff --git a/heron/stmgr/tests/cpp/server/stateful-restorer_unittest.cpp b/heron/stmgr/tests/cpp/server/stateful-restorer_unittest.cpp
index 69b44d0..783ce06 100644
--- a/heron/stmgr/tests/cpp/server/stateful-restorer_unittest.cpp
+++ b/heron/stmgr/tests/cpp/server/stateful-restorer_unittest.cpp
@@ -346,7 +346,7 @@ TEST(StatefulRestorer, deadinstances) {
   // Send notification that some tasks have recovered
   EXPECT_GT(local_tasks.size(), 1);
   bool first = true;
-  int32_t troublesome_task;
+  int32_t troublesome_task = 0;
   for (auto task : local_tasks) {
     if (first) {
       first = false;
@@ -426,7 +426,7 @@ TEST(StatefulRestorer, deadckptmgr) {
   // Send notification that some tasks have recovered
   EXPECT_GT(local_tasks.size(), 1);
   bool first = true;
-  int32_t troublesome_task;
+  int32_t troublesome_task = 0;
   // ckpt delivers some checkpoints
   for (auto task : local_tasks) {
     if (first) {
diff --git a/maven_install.json b/maven_install.json
index ce48939..4a8c29f 100644
--- a/maven_install.json
+++ b/maven_install.json
@@ -1,140 +1,132 @@
 {
     "dependency_tree": {
+        "__AUTOGENERATED_FILE_DO_NOT_MODIFY_THIS_FILE_MANUALLY": 554853469,
         "conflict_resolution": {},
         "dependencies": [
             {
                 "coord": "ant:ant:1.6.5",
-                "file": "v1/https/jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5.jar",
                     "https://maven.google.com/ant/ant/1.6.5/ant-1.6.5.jar",
                     "https://repo1.maven.org/maven2/ant/ant/1.6.5/ant-1.6.5.jar"
                 ],
-                "sha256": "f06a601c718a7c9262d74b7ec3baad14c82584e89235089b4f821d6a44d9e1e4"
+                "sha256": "f06a601c718a7c9262d74b7ec3baad14c82584e89235089b4f821d6a44d9e1e4",
+                "url": "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5.jar"
             },
             {
                 "coord": "ant:ant:jar:sources:1.6.5",
-                "file": "v1/https/jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5-sources.jar",
                     "https://maven.google.com/ant/ant/1.6.5/ant-1.6.5-sources.jar",
                     "https://repo1.maven.org/maven2/ant/ant/1.6.5/ant-1.6.5-sources.jar"
                 ],
-                "sha256": "45369155a1ecca333ae193761094cba1fe279bdf537ed70d2bb968b7c7797ce2"
+                "sha256": "45369155a1ecca333ae193761094cba1fe279bdf537ed70d2bb968b7c7797ce2",
+                "url": "https://jcenter.bintray.com/ant/ant/1.6.5/ant-1.6.5-sources.jar"
             },
             {
                 "coord": "antlr:antlr:2.7.7",
-                "file": "v1/https/jcenter.bintray.com/antlr/antlr/2.7.7/antlr-2.7.7.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/antlr/antlr/2.7.7/antlr-2.7.7.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/antlr/antlr/2.7.7/antlr-2.7.7.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/antlr/antlr/2.7.7/antlr-2.7.7.jar",
                     "https://maven.google.com/antlr/antlr/2.7.7/antlr-2.7.7.jar",
                     "https://repo1.maven.org/maven2/antlr/antlr/2.7.7/antlr-2.7.7.jar"
                 ],
-                "sha256": "88fbda4b912596b9f56e8e12e580cc954bacfb51776ecfddd3e18fc1cf56dc4c"
+                "sha256": "88fbda4b912596b9f56e8e12e580cc954bacfb51776ecfddd3e18fc1cf56dc4c",
+                "url": "https://jcenter.bintray.com/antlr/antlr/2.7.7/antlr-2.7.7.jar"
             },
             {
                 "coord": "antlr:antlr:jar:sources:2.7.7",
-                "file": null,
+                "dependencies": [],
                 "directDependencies": [],
-                "dependencies": []
+                "file": null
             },
             {
                 "coord": "aopalliance:aopalliance:1.0",
-                "file": "v1/https/jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar",
                     "https://maven.google.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar",
                     "https://repo1.maven.org/maven2/aopalliance/aopalliance/1.0/aopalliance-1.0.jar"
                 ],
-                "sha256": "0addec670fedcd3f113c5c8091d783280d23f75e3acb841b61a9cdb079376a08"
+                "sha256": "0addec670fedcd3f113c5c8091d783280d23f75e3acb841b61a9cdb079376a08",
+                "url": "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0.jar"
             },
             {
                 "coord": "aopalliance:aopalliance:jar:sources:1.0",
-                "file": "v1/https/jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar",
                     "https://maven.google.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar",
                     "https://repo1.maven.org/maven2/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar"
                 ],
-                "sha256": "e6ef91d439ada9045f419c77543ebe0416c3cdfc5b063448343417a3e4a72123"
+                "sha256": "e6ef91d439ada9045f419c77543ebe0416c3cdfc5b063448343417a3e4a72123",
+                "url": "https://jcenter.bintray.com/aopalliance/aopalliance/1.0/aopalliance-1.0-sources.jar"
             },
             {
                 "coord": "cglib:cglib:3.1",
-                "file": "v1/https/jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1.jar",
+                "file": "v1/https/jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1.jar",
                     "https://maven.google.com/cglib/cglib/3.1/cglib-3.1.jar",
                     "https://repo1.maven.org/maven2/cglib/cglib/3.1/cglib-3.1.jar"
                 ],
-                "sha256": "175c86146a6b1eefc891df9e99aef171cfd64998ba1c01c5045de56aef9778cb"
+                "sha256": "175c86146a6b1eefc891df9e99aef171cfd64998ba1c01c5045de56aef9778cb",
+                "url": "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1.jar"
             },
             {
                 "coord": "cglib:cglib:jar:sources:3.1",
-                "file": "v1/https/jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1-sources.jar",
                     "https://maven.google.com/cglib/cglib/3.1/cglib-3.1-sources.jar",
                     "https://repo1.maven.org/maven2/cglib/cglib/3.1/cglib-3.1-sources.jar"
                 ],
-                "sha256": "386f9a55ae8fed3cf8014f2c20344ce9ce05cd2e4fa467e5e4d8027abb3d62eb"
+                "sha256": "386f9a55ae8fed3cf8014f2c20344ce9ce05cd2e4fa467e5e4d8027abb3d62eb",
+                "url": "https://jcenter.bintray.com/cglib/cglib/3.1/cglib-3.1-sources.jar"
             },
             {
                 "coord": "classworlds:classworlds:1.1-alpha-2",
-                "file": "v1/https/jcenter.bintray.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar",
-                "directDependencies": [],
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar",
+                "file": "v1/https/jcenter.bintray.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar",
                     "https://maven.google.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar",
                     "https://repo1.maven.org/maven2/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar"
                 ],
-                "sha256": "2bf4e59f3acd106fea6145a9a88fe8956509f8b9c0fdd11eb96fee757269e3f3"
+                "sha256": "2bf4e59f3acd106fea6145a9a88fe8956509f8b9c0fdd11eb96fee757269e3f3",
+                "url": "https://jcenter.bintray.com/classworlds/classworlds/1.1-alpha-2/classworlds-1.1-alpha-2.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-core:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar",
-                "directDependencies": [
-                    "joda-time:joda-time:2.9.3",
-                    "commons-logging:commons-logging:1.2",
-                    "software.amazon.ion:ion-java:1.0.1",
-                    "com.fasterxml.jackson.core:jackson-databind:jar:2.8.8",
-                    "org.apache.httpcomponents:httpclient:4.5.2",
-                    "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.6.6"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "joda-time:joda-time:2.9.3",
@@ -147,52 +139,56 @@
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar",
+                "directDependencies": [
+                    "joda-time:joda-time:2.9.3",
+                    "commons-logging:commons-logging:1.2",
+                    "software.amazon.ion:ion-java:1.0.1",
+                    "com.fasterxml.jackson.core:jackson-databind:jar:2.8.8",
+                    "org.apache.httpcomponents:httpclient:4.5.2",
+                    "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.6.6"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar"
                 ],
-                "sha256": "da26fdfe4c119f8a88a964368572945083ce01432d4ee04a8ac6173aa203f677"
+                "sha256": "da26fdfe4c119f8a88a964368572945083ce01432d4ee04a8ac6173aa203f677",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-core:jar:sources:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "joda-time:joda-time:jar:sources:2.9.3",
                     "software.amazon.ion:ion-java:jar:sources:1.0.1",
                     "commons-logging:commons-logging:jar:sources:1.2",
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
                     "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:sources:2.6.6",
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
+                    "commons-codec:commons-codec:jar:sources:1.11",
+                    "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "joda-time:joda-time:jar:sources:2.9.3",
                     "software.amazon.ion:ion-java:jar:sources:1.0.1",
                     "commons-logging:commons-logging:jar:sources:1.2",
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
                     "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:sources:2.6.6",
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
-                    "commons-codec:commons-codec:jar:sources:1.11",
-                    "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar"
                 ],
-                "sha256": "9eb941d2c626ba27a66af7ce30b4d145621c405ced22fc4452f7e330fdce38f8"
+                "sha256": "9eb941d2c626ba27a66af7ce30b4d145621c405ced22fc4452f7e330fdce38f8",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-core/1.11.58/aws-java-sdk-core-1.11.58-sources.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-kms:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar",
-                "directDependencies": [
-                    "com.amazonaws:aws-java-sdk-core:1.11.58",
-                    "com.amazonaws:jmespath-java:1.11.58"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "joda-time:joda-time:2.9.3",
@@ -208,21 +204,21 @@
                     "com.amazonaws:aws-java-sdk-core:1.11.58",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar",
+                "directDependencies": [
+                    "com.amazonaws:aws-java-sdk-core:1.11.58",
+                    "com.amazonaws:jmespath-java:1.11.58"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar"
                 ],
-                "sha256": "2ce68f26b2da2bd17c6f2b5943cdf67c1c4e6350efee9bf44f6d4e7436db1dca"
+                "sha256": "2ce68f26b2da2bd17c6f2b5943cdf67c1c4e6350efee9bf44f6d4e7436db1dca",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-kms:jar:sources:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar",
-                "directDependencies": [
-                    "com.amazonaws:aws-java-sdk-core:jar:sources:1.11.58",
-                    "com.amazonaws:jmespath-java:jar:sources:1.11.58"
-                ],
                 "dependencies": [
                     "joda-time:joda-time:jar:sources:2.9.3",
                     "com.amazonaws:jmespath-java:jar:sources:1.11.58",
@@ -237,22 +233,21 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar",
+                "directDependencies": [
+                    "com.amazonaws:aws-java-sdk-core:jar:sources:1.11.58",
+                    "com.amazonaws:jmespath-java:jar:sources:1.11.58"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar"
                 ],
-                "sha256": "a908f280b49ae943e35d22c96b606a328e8d2020a3757e4a89bc0b3648b5bc26"
+                "sha256": "a908f280b49ae943e35d22c96b606a328e8d2020a3757e4a89bc0b3648b5bc26",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-kms/1.11.58/aws-java-sdk-kms-1.11.58-sources.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-s3:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar",
-                "directDependencies": [
-                    "com.amazonaws:aws-java-sdk-core:1.11.58",
-                    "com.amazonaws:aws-java-sdk-kms:1.11.58",
-                    "com.amazonaws:jmespath-java:1.11.58"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "joda-time:joda-time:2.9.3",
@@ -269,22 +264,22 @@
                     "com.amazonaws:aws-java-sdk-core:1.11.58",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar",
+                "directDependencies": [
+                    "com.amazonaws:aws-java-sdk-core:1.11.58",
+                    "com.amazonaws:aws-java-sdk-kms:1.11.58",
+                    "com.amazonaws:jmespath-java:1.11.58"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar"
                 ],
-                "sha256": "e556c684b1030e05b6eff23b47e8ce7da2970c3a1056c6d0bbd4998d133150d0"
+                "sha256": "e556c684b1030e05b6eff23b47e8ce7da2970c3a1056c6d0bbd4998d133150d0",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58.jar"
             },
             {
                 "coord": "com.amazonaws:aws-java-sdk-s3:jar:sources:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar",
-                "directDependencies": [
-                    "com.amazonaws:aws-java-sdk-core:jar:sources:1.11.58",
-                    "com.amazonaws:aws-java-sdk-kms:jar:sources:1.11.58",
-                    "com.amazonaws:jmespath-java:jar:sources:1.11.58"
-                ],
                 "dependencies": [
                     "joda-time:joda-time:jar:sources:2.9.3",
                     "com.amazonaws:jmespath-java:jar:sources:1.11.58",
@@ -300,355 +295,354 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar",
+                "directDependencies": [
+                    "com.amazonaws:aws-java-sdk-core:jar:sources:1.11.58",
+                    "com.amazonaws:aws-java-sdk-kms:jar:sources:1.11.58",
+                    "com.amazonaws:jmespath-java:jar:sources:1.11.58"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar",
                     "https://maven.google.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar"
                 ],
-                "sha256": "52bd56c1c7560f6d98ba5c1da8146a4cf014b0a99825c0bc1caea476ccb306bb"
+                "sha256": "52bd56c1c7560f6d98ba5c1da8146a4cf014b0a99825c0bc1caea476ccb306bb",
+                "url": "https://jcenter.bintray.com/com/amazonaws/aws-java-sdk-s3/1.11.58/aws-java-sdk-s3-1.11.58-sources.jar"
             },
             {
                 "coord": "com.amazonaws:jmespath-java:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar",
                     "https://maven.google.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar"
                 ],
-                "sha256": "b37aef278ae79e2b48e049edde5ff2298223e24180f7dd195e77587d445678e4"
+                "sha256": "b37aef278ae79e2b48e049edde5ff2298223e24180f7dd195e77587d445678e4",
+                "url": "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58.jar"
             },
             {
                 "coord": "com.amazonaws:jmespath-java:jar:sources:1.11.58",
-                "file": "v1/https/jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar",
                     "https://maven.google.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar",
                     "https://repo1.maven.org/maven2/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar"
                 ],
-                "sha256": "40e612696456351f2c5e1cd039f3f109027a56627b967e900f84ff1d35d2e910"
+                "sha256": "40e612696456351f2c5e1cd039f3f109027a56627b967e900f84ff1d35d2e910",
+                "url": "https://jcenter.bintray.com/com/amazonaws/jmespath-java/1.11.58/jmespath-java-1.11.58-sources.jar"
             },
             {
                 "coord": "com.beust:jcommander:1.48",
-                "file": "v1/https/jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48.jar",
                     "https://maven.google.com/com/beust/jcommander/1.48/jcommander-1.48.jar",
                     "https://repo1.maven.org/maven2/com/beust/jcommander/1.48/jcommander-1.48.jar"
                 ],
-                "sha256": "a7313fcfde070930e40ec79edf3c5948cf34e4f0d25cb3a09f9963d8bdd84113"
+                "sha256": "a7313fcfde070930e40ec79edf3c5948cf34e4f0d25cb3a09f9963d8bdd84113",
+                "url": "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48.jar"
             },
             {
                 "coord": "com.beust:jcommander:jar:sources:1.48",
-                "file": "v1/https/jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar",
                     "https://maven.google.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar",
                     "https://repo1.maven.org/maven2/com/beust/jcommander/1.48/jcommander-1.48-sources.jar"
                 ],
-                "sha256": "034401849e035a879dbd4c1937249a7d0c7a61c54bf9e668187718e56770ecb3"
+                "sha256": "034401849e035a879dbd4c1937249a7d0c7a61c54bf9e668187718e56770ecb3",
+                "url": "https://jcenter.bintray.com/com/beust/jcommander/1.48/jcommander-1.48-sources.jar"
             },
             {
                 "coord": "com.esotericsoftware:kryo:3.0.3",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar",
-                "directDependencies": [
-                    "com.esotericsoftware:minlog:1.3.0",
-                    "com.esotericsoftware:reflectasm:1.10.1",
-                    "org.objenesis:objenesis:2.1"
-                ],
                 "dependencies": [
                     "com.esotericsoftware:minlog:1.3.0",
                     "org.objenesis:objenesis:2.1",
                     "org.ow2.asm:asm:5.0.4",
                     "com.esotericsoftware:reflectasm:1.10.1"
                 ],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar",
+                "directDependencies": [
+                    "com.esotericsoftware:minlog:1.3.0",
+                    "com.esotericsoftware:reflectasm:1.10.1",
+                    "org.objenesis:objenesis:2.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar",
                     "https://maven.google.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar"
                 ],
-                "sha256": "5c295b23480225ff6e7d6770dfa904bedcec8556c27234fea0a271fe13195f69"
+                "sha256": "5c295b23480225ff6e7d6770dfa904bedcec8556c27234fea0a271fe13195f69",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3.jar"
             },
             {
                 "coord": "com.esotericsoftware:kryo:jar:sources:3.0.3",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar",
-                "directDependencies": [
-                    "com.esotericsoftware:minlog:jar:sources:1.3.0",
-                    "com.esotericsoftware:reflectasm:jar:sources:1.10.1",
-                    "org.objenesis:objenesis:jar:sources:2.1"
-                ],
                 "dependencies": [
                     "com.esotericsoftware:minlog:jar:sources:1.3.0",
                     "com.esotericsoftware:reflectasm:jar:sources:1.10.1",
                     "org.objenesis:objenesis:jar:sources:2.1",
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar",
+                "directDependencies": [
+                    "com.esotericsoftware:minlog:jar:sources:1.3.0",
+                    "com.esotericsoftware:reflectasm:jar:sources:1.10.1",
+                    "org.objenesis:objenesis:jar:sources:2.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar",
                     "https://maven.google.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar"
                 ],
-                "sha256": "5a02ca261f78e21ad030e5f8ac5dc61bf93160128e53d98ef0be41dbe6dbf4a6"
+                "sha256": "5a02ca261f78e21ad030e5f8ac5dc61bf93160128e53d98ef0be41dbe6dbf4a6",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/kryo/3.0.3/kryo-3.0.3-sources.jar"
             },
             {
                 "coord": "com.esotericsoftware:minlog:1.3.0",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar",
                     "https://maven.google.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar"
                 ],
-                "sha256": "f7b399d3a5478a4f3e0d98bd1c9f47766119c66414bc33aa0f6cde0066f24cc2"
+                "sha256": "f7b399d3a5478a4f3e0d98bd1c9f47766119c66414bc33aa0f6cde0066f24cc2",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar"
             },
             {
                 "coord": "com.esotericsoftware:minlog:jar:sources:1.3.0",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar",
                     "https://maven.google.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar"
                 ],
-                "sha256": "99872e1e68874771d77bf3131620bf656b541fa993d3f6e9d29b9f03ae423d17"
+                "sha256": "99872e1e68874771d77bf3131620bf656b541fa993d3f6e9d29b9f03ae423d17",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0-sources.jar"
             },
             {
                 "coord": "com.esotericsoftware:reflectasm:1.10.1",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar",
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar",
                     "https://maven.google.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar"
                 ],
-                "sha256": "94867942701229aead2b99cfbfdb68ea0afe4ec6fb943e5a10cb875f6e94f5b4"
+                "sha256": "94867942701229aead2b99cfbfdb68ea0afe4ec6fb943e5a10cb875f6e94f5b4",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1.jar"
             },
             {
                 "coord": "com.esotericsoftware:reflectasm:jar:sources:1.10.1",
-                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar",
                     "https://maven.google.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar",
                     "https://repo1.maven.org/maven2/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar"
                 ],
-                "sha256": "5d46267d7130e2d5963472dbb2a975ae764ec9f616c1c9b7e7961fcc600d38a0"
+                "sha256": "5d46267d7130e2d5963472dbb2a975ae764ec9f616c1c9b7e7961fcc600d38a0",
+                "url": "https://jcenter.bintray.com/com/esotericsoftware/reflectasm/1.10.1/reflectasm-1.10.1-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar"
                 ],
-                "sha256": "1ff7b1c91658506f1050b39d1564eb4d5dc63586dd709bad58428a63775d75a8"
+                "sha256": "1ff7b1c91658506f1050b39d1564eb4d5dc63586dd709bad58428a63775d75a8",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar"
                 ],
-                "sha256": "fc02f7412728e7055c68e996ffcf268442388aa38cf176b12e5801b5d264aa6a"
+                "sha256": "fc02f7412728e7055c68e996ffcf268442388aa38cf176b12e5801b5d264aa6a",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-annotations/2.8.8/jackson-annotations-2.8.8-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-core:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar"
                 ],
-                "sha256": "d9bde8c72c22202bf17b05c7811db4964ff8e843d97c00a9bfb048c0fe7a726b"
+                "sha256": "d9bde8c72c22202bf17b05c7811db4964ff8e843d97c00a9bfb048c0fe7a726b",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar"
                 ],
-                "sha256": "eea7f4ee3e67a72b3cf9825ac425787fe14d23368e9946f991f9e6ba4509733b"
+                "sha256": "eea7f4ee3e67a72b3cf9825ac425787fe14d23368e9946f991f9e6ba4509733b",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-core/2.8.8/jackson-core-2.8.8-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-databind:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar",
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar"
                 ],
-                "sha256": "bd2959a21974cb361cea6a9295b6e8600e8b6a8c866a768d22b952016bce3248"
+                "sha256": "bd2959a21974cb361cea6a9295b6e8600e8b6a8c866a768d22b952016bce3248",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar"
                 ],
-                "sha256": "2e1f83bf79b2207d7fc9063a05467371358f9fc282ba0728494b0e2dc5ea94f1"
+                "sha256": "2e1f83bf79b2207d7fc9063a05467371358f9fc282ba0728494b0e2dc5ea94f1",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/core/jackson-databind/2.8.8/jackson-databind-2.8.8-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.6.6",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar",
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar",
                     "https://maven.google.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar"
                 ],
-                "sha256": "68edce58a340c89e14158f13291f8b6f9534f7cc102f44f9440cc9dd03f709b0"
+                "sha256": "68edce58a340c89e14158f13291f8b6f9534f7cc102f44f9440cc9dd03f709b0",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:sources:2.6.6",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar"
                 ],
-                "sha256": "101fe6af52c0fd1cfb2a8bc4348113a36c71b392adf716d557f009ca96dcf920"
+                "sha256": "101fe6af52c0fd1cfb2a8bc4348113a36c71b392adf716d557f009ca96dcf920",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.6.6/jackson-dataformat-cbor-2.6.6-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar",
                     "https://maven.google.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar"
                 ],
-                "sha256": "7c3b49333047f07a24fbdd1c121e7f36956f4865dff53c8c64f9897fe629cebb"
+                "sha256": "7c3b49333047f07a24fbdd1c121e7f36956f4865dff53c8c64f9897fe629cebb",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar"
                 ],
-                "sha256": "9867aad22ad8a47f078211355995cb0b6d590627fb88e9535778bda0d1b28254"
+                "sha256": "9867aad22ad8a47f078211355995cb0b6d590627fb88e9535778bda0d1b28254",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.8.8/jackson-jaxrs-base-2.8.8-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:2.8.8",
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
-                    "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
@@ -656,23 +650,23 @@
                     "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
-                "mirror_urls": [
-                    "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
-                    "https://maven.google.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
-                    "https://repo1.maven.org/maven2/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar"
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:2.8.8",
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
+                    "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
+                "mirror_urls": [
+                    "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
+                    "https://maven.google.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar",
+                    "https://repo1.maven.org/maven2/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar"
                 ],
-                "sha256": "50638f0809a551ca658878a606b6eab50e3e29deb59bb2f87fecc0fe4ef13ae5"
+                "sha256": "50638f0809a551ca658878a606b6eab50e3e29deb59bb2f87fecc0fe4ef13ae5",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar:sources:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
@@ -680,90 +674,91 @@
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar:sources:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar"
                 ],
-                "sha256": "af19e6af05ca54f9ec49b25bbc88761d4e743bd907564a797092428b122ac964"
+                "sha256": "af19e6af05ca54f9ec49b25bbc88761d4e743bd907564a797092428b122ac964",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.8.8/jackson-jaxrs-json-provider-2.8.8-sources.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar",
                     "https://maven.google.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar"
                 ],
-                "sha256": "ff3747b94fe0be4d378dab0d5c8e7baad77a3b4f59aac19809908626e92a8122"
+                "sha256": "ff3747b94fe0be4d378dab0d5c8e7baad77a3b4f59aac19809908626e92a8122",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8.jar"
             },
             {
                 "coord": "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar:sources:2.8.8",
-                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar",
                     "https://maven.google.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar"
                 ],
-                "sha256": "dd9da5507c59f9a3315a45b22694a46967a1668c7704181b0c3d80475f59c7bd"
+                "sha256": "dd9da5507c59f9a3315a45b22694a46967a1668c7704181b0c3d80475f59c7bd",
+                "url": "https://jcenter.bintray.com/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.8.8/jackson-module-jaxb-annotations-2.8.8-sources.jar"
             },
             {
                 "coord": "com.github.luben:zstd-jni:1.3.8-1",
-                "file": "v1/https/jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar",
                     "https://maven.google.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar",
                     "https://repo1.maven.org/maven2/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar"
                 ],
-                "sha256": "05ab236e5beda6549ff14b2d3e604b68da6bac925bb0dcdb53ebc5a3ced2856c"
+                "sha256": "05ab236e5beda6549ff14b2d3e604b68da6bac925bb0dcdb53ebc5a3ced2856c",
+                "url": "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1.jar"
             },
             {
                 "coord": "com.github.luben:zstd-jni:jar:sources:1.3.8-1",
-                "file": "v1/https/jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar",
                     "https://maven.google.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar",
                     "https://repo1.maven.org/maven2/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar"
                 ],
-                "sha256": "1281edb1b55520ea959b1390229baa36bb879f1c0cf34a863d3784af293c7d80"
+                "sha256": "1281edb1b55520ea959b1390229baa36bb879f1c0cf34a863d3784af293c7d80",
+                "url": "https://jcenter.bintray.com/com/github/luben/zstd-jni/1.3.8-1/zstd-jni-1.3.8-1-sources.jar"
             },
             {
                 "coord": "com.google.api-client:google-api-client:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar",
-                "directDependencies": [
-                    "com.google.guava:guava-jdk5:17.0",
-                    "com.google.http-client:google-http-client-jackson2:1.22.0",
-                    "com.google.oauth-client:google-oauth-client:1.22.0"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
@@ -776,22 +771,22 @@
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar",
+                "directDependencies": [
+                    "com.google.guava:guava-jdk5:17.0",
+                    "com.google.http-client:google-http-client-jackson2:1.22.0",
+                    "com.google.oauth-client:google-oauth-client:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar",
                     "https://maven.google.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar",
                     "https://repo1.maven.org/maven2/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar"
                 ],
-                "sha256": "47c625c83a8cf97b8bbdff2acde923ff8fd3174e62aabcfc5d1b86692594ffba"
+                "sha256": "47c625c83a8cf97b8bbdff2acde923ff8fd3174e62aabcfc5d1b86692594ffba",
+                "url": "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0.jar"
             },
             {
                 "coord": "com.google.api-client:google-api-client:jar:sources:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar",
-                "directDependencies": [
-                    "com.google.guava:guava-jdk5:jar:sources:17.0",
-                    "com.google.http-client:google-http-client-jackson2:jar:sources:1.22.0",
-                    "com.google.oauth-client:google-oauth-client:jar:sources:1.22.0"
-                ],
                 "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "com.google.oauth-client:google-oauth-client:jar:sources:1.22.0",
@@ -804,20 +799,22 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar",
+                "directDependencies": [
+                    "com.google.guava:guava-jdk5:jar:sources:17.0",
+                    "com.google.http-client:google-http-client-jackson2:jar:sources:1.22.0",
+                    "com.google.oauth-client:google-oauth-client:jar:sources:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar",
                     "https://maven.google.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar"
                 ],
-                "sha256": "d6d529741964e7d3b42923b352bd07ea351726d85a257e66cc8ffd8af276984a"
+                "sha256": "d6d529741964e7d3b42923b352bd07ea351726d85a257e66cc8ffd8af276984a",
+                "url": "https://jcenter.bintray.com/com/google/api-client/google-api-client/1.22.0/google-api-client-1.22.0-sources.jar"
             },
             {
                 "coord": "com.google.apis:google-api-services-storage:jar:sources:v1-rev108-1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar",
-                "directDependencies": [
-                    "com.google.api-client:google-api-client:jar:sources:1.22.0"
-                ],
                 "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "com.google.oauth-client:google-oauth-client:jar:sources:1.22.0",
@@ -831,20 +828,20 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar",
+                "directDependencies": [
+                    "com.google.api-client:google-api-client:jar:sources:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar",
                     "https://maven.google.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar"
                 ],
-                "sha256": "19414dced22793d0ba423ac43059e2a3af17ac8df9e2dbc269b62a712490235d"
+                "sha256": "19414dced22793d0ba423ac43059e2a3af17ac8df9e2dbc269b62a712490235d",
+                "url": "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0-sources.jar"
             },
             {
                 "coord": "com.google.apis:google-api-services-storage:v1-rev108-1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar",
-                "directDependencies": [
-                    "com.google.api-client:google-api-client:1.22.0"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
@@ -858,151 +855,150 @@
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar",
+                "directDependencies": [
+                    "com.google.api-client:google-api-client:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar",
                     "https://maven.google.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar",
                     "https://repo1.maven.org/maven2/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar"
                 ],
-                "sha256": "eae26d9589afb5fcd4f5db18ce2b7ef2aa10a333a97da9056e22c0fc203d6cad"
+                "sha256": "eae26d9589afb5fcd4f5db18ce2b7ef2aa10a333a97da9056e22c0fc203d6cad",
+                "url": "https://jcenter.bintray.com/com/google/apis/google-api-services-storage/v1-rev108-1.22.0/google-api-services-storage-v1-rev108-1.22.0.jar"
             },
             {
                 "coord": "com.google.code.findbugs:jsr305:3.0.2",
-                "file": "v1/https/jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar",
                     "https://maven.google.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar",
                     "https://repo1.maven.org/maven2/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar"
                 ],
-                "sha256": "766ad2a0783f2687962c8ad74ceecc38a28b9f72a2d085ee438b7813e928d0c7"
+                "sha256": "766ad2a0783f2687962c8ad74ceecc38a28b9f72a2d085ee438b7813e928d0c7",
+                "url": "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar"
             },
             {
                 "coord": "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
-                "file": "v1/https/jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar",
                     "https://maven.google.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar"
                 ],
-                "sha256": "1c9e85e272d0708c6a591dc74828c71603053b48cc75ae83cce56912a2aa063b"
+                "sha256": "1c9e85e272d0708c6a591dc74828c71603053b48cc75ae83cce56912a2aa063b",
+                "url": "https://jcenter.bintray.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2-sources.jar"
             },
             {
                 "coord": "com.google.code.gson:gson:2.8.2",
-                "file": "v1/https/jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar",
                     "https://maven.google.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar",
                     "https://repo1.maven.org/maven2/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar"
                 ],
-                "sha256": "b7134929f7cc7c04021ec1cc27ef63ab907e410cf0588e397b8851181eb91092"
+                "sha256": "b7134929f7cc7c04021ec1cc27ef63ab907e410cf0588e397b8851181eb91092",
+                "url": "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2.jar"
             },
             {
                 "coord": "com.google.code.gson:gson:jar:sources:2.8.2",
-                "file": "v1/https/jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar",
                     "https://maven.google.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar"
                 ],
-                "sha256": "1c291a2fe0867d66ef86832e014889a398a5c5b8e823206324a782b212df0df3"
+                "sha256": "1c291a2fe0867d66ef86832e014889a398a5c5b8e823206324a782b212df0df3",
+                "url": "https://jcenter.bintray.com/com/google/code/gson/gson/2.8.2/gson-2.8.2-sources.jar"
             },
             {
                 "coord": "com.google.collections:google-collections:1.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar",
                     "https://maven.google.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar",
                     "https://repo1.maven.org/maven2/com/google/collections/google-collections/1.0/google-collections-1.0.jar"
                 ],
-                "sha256": "81b8d638af0083c4b877099d56aa0fee714485cd2ace1b6a09cab867cadb375d"
+                "sha256": "81b8d638af0083c4b877099d56aa0fee714485cd2ace1b6a09cab867cadb375d",
+                "url": "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0.jar"
             },
             {
                 "coord": "com.google.collections:google-collections:jar:sources:1.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar",
                     "https://maven.google.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar"
                 ],
-                "sha256": "dbb1a31cbbbaf5596cd7431a551cada2c329bba53b2f76900af35ab17d307f21"
+                "sha256": "dbb1a31cbbbaf5596cd7431a551cada2c329bba53b2f76900af35ab17d307f21",
+                "url": "https://jcenter.bintray.com/com/google/collections/google-collections/1.0/google-collections-1.0-sources.jar"
             },
             {
                 "coord": "com.google.guava:guava-jdk5:17.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar",
                     "https://maven.google.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar",
                     "https://repo1.maven.org/maven2/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar"
                 ],
-                "sha256": "59bf456547b6dda3ced968cb56f7f2d3e14474e2de2960a32c47e31c1e456c61"
+                "sha256": "59bf456547b6dda3ced968cb56f7f2d3e14474e2de2960a32c47e31c1e456c61",
+                "url": "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0.jar"
             },
             {
                 "coord": "com.google.guava:guava-jdk5:jar:sources:17.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar",
                     "https://maven.google.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar"
                 ],
-                "sha256": "70777456f41c7cf275e6e51a3b20a7cdf2f19efc8e7704b65275da38ad9cb9a4"
+                "sha256": "70777456f41c7cf275e6e51a3b20a7cdf2f19efc8e7704b65275da38ad9cb9a4",
+                "url": "https://jcenter.bintray.com/com/google/guava/guava-jdk5/17.0/guava-jdk5-17.0-sources.jar"
             },
             {
                 "coord": "com.google.guava:guava:18.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0.jar",
                     "https://maven.google.com/com/google/guava/guava/18.0/guava-18.0.jar",
                     "https://repo1.maven.org/maven2/com/google/guava/guava/18.0/guava-18.0.jar"
                 ],
-                "sha256": "d664fbfc03d2e5ce9cab2a44fb01f1d0bf9dfebeccc1a473b1f9ea31f79f6f99"
+                "sha256": "d664fbfc03d2e5ce9cab2a44fb01f1d0bf9dfebeccc1a473b1f9ea31f79f6f99",
+                "url": "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0.jar"
             },
             {
                 "coord": "com.google.guava:guava:jar:sources:18.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0-sources.jar",
                     "https://maven.google.com/com/google/guava/guava/18.0/guava-18.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/guava/guava/18.0/guava-18.0-sources.jar"
                 ],
-                "sha256": "0f3381ce4caa3a01602cdf0d0f4437b9ad8f347ddabbd873ded4d498e19d9590"
+                "sha256": "0f3381ce4caa3a01602cdf0d0f4437b9ad8f347ddabbd873ded4d498e19d9590",
+                "url": "https://jcenter.bintray.com/com/google/guava/guava/18.0/guava-18.0-sources.jar"
             },
             {
                 "coord": "com.google.http-client:google-http-client-jackson2:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
-                    "com.google.http-client:google-http-client:1.22.0"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
@@ -1012,21 +1008,21 @@
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
+                    "com.google.http-client:google-http-client:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar",
                     "https://maven.google.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar",
                     "https://repo1.maven.org/maven2/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar"
                 ],
-                "sha256": "45b1e34b2dcef5cb496ef25a1223d19cf102b8c2ea4abf96491631b2faf4611c"
+                "sha256": "45b1e34b2dcef5cb496ef25a1223d19cf102b8c2ea4abf96491631b2faf4611c",
+                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0.jar"
             },
             {
                 "coord": "com.google.http-client:google-http-client-jackson2:jar:sources:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                    "com.google.http-client:google-http-client:jar:sources:1.22.0"
-                ],
                 "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "commons-logging:commons-logging:jar:sources:1.2",
@@ -1036,21 +1032,21 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
+                    "com.google.http-client:google-http-client:jar:sources:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar",
                     "https://maven.google.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar"
                 ],
-                "sha256": "76884259004572d24a669aed09bd35ddbd8e6155a2a38dfd96e61f4690baa00f"
+                "sha256": "76884259004572d24a669aed09bd35ddbd8e6155a2a38dfd96e61f4690baa00f",
+                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client-jackson2/1.22.0/google-http-client-jackson2-1.22.0-sources.jar"
             },
             {
                 "coord": "com.google.http-client:google-http-client:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar",
-                "directDependencies": [
-                    "com.google.code.findbugs:jsr305:3.0.2",
-                    "org.apache.httpcomponents:httpclient:4.5.2"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
@@ -1058,21 +1054,21 @@
                     "org.apache.httpcomponents:httpclient:4.5.2",
                     "org.apache.httpcomponents:httpcore:4.4.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar",
+                "directDependencies": [
+                    "com.google.code.findbugs:jsr305:3.0.2",
+                    "org.apache.httpcomponents:httpclient:4.5.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar",
                     "https://maven.google.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar",
                     "https://repo1.maven.org/maven2/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar"
                 ],
-                "sha256": "f88ffa329ac52fb4f2ff0eb877ef7318423ac9b791a107f886ed5c7a00e77e11"
+                "sha256": "f88ffa329ac52fb4f2ff0eb877ef7318423ac9b791a107f886ed5c7a00e77e11",
+                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0.jar"
             },
             {
                 "coord": "com.google.http-client:google-http-client:jar:sources:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar",
-                "directDependencies": [
-                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
-                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
-                ],
                 "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "commons-logging:commons-logging:jar:sources:1.2",
@@ -1080,103 +1076,103 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar",
+                "directDependencies": [
+                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
+                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar",
                     "https://maven.google.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar"
                 ],
-                "sha256": "7da8c0ed2203e84cf8c2248af98de177348dfeac92fc032e7aa220e78f2f80ba"
+                "sha256": "7da8c0ed2203e84cf8c2248af98de177348dfeac92fc032e7aa220e78f2f80ba",
+                "url": "https://jcenter.bintray.com/com/google/http-client/google-http-client/1.22.0/google-http-client-1.22.0-sources.jar"
             },
             {
                 "coord": "com.google.inject.extensions:guice-assistedinject:4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar",
-                "directDependencies": [
-                    "com.google.inject:guice:4.0"
-                ],
                 "dependencies": [
                     "com.google.guava:guava:18.0",
                     "aopalliance:aopalliance:1.0",
                     "javax.inject:javax.inject:1",
                     "com.google.inject:guice:4.0"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar",
+                "directDependencies": [
+                    "com.google.inject:guice:4.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar",
                     "https://maven.google.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar",
                     "https://repo1.maven.org/maven2/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar"
                 ],
-                "sha256": "ed44e4d809e6b3bc41ccd6e8b5acf43fb517234f4ac4c1125d7a82c44826f147"
+                "sha256": "ed44e4d809e6b3bc41ccd6e8b5acf43fb517234f4ac4c1125d7a82c44826f147",
+                "url": "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0.jar"
             },
             {
                 "coord": "com.google.inject.extensions:guice-assistedinject:jar:sources:4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar",
-                "directDependencies": [
-                    "com.google.inject:guice:jar:sources:4.0"
-                ],
                 "dependencies": [
                     "aopalliance:aopalliance:jar:sources:1.0",
                     "com.google.inject:guice:jar:sources:4.0",
                     "javax.inject:javax.inject:jar:sources:1",
                     "com.google.guava:guava:jar:sources:18.0"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar",
+                "directDependencies": [
+                    "com.google.inject:guice:jar:sources:4.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar",
                     "https://maven.google.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar"
                 ],
-                "sha256": "689d2ee112437e3b03082876a902bd228f6067568d7d31c02b31a22babc889f3"
+                "sha256": "689d2ee112437e3b03082876a902bd228f6067568d7d31c02b31a22babc889f3",
+                "url": "https://jcenter.bintray.com/com/google/inject/extensions/guice-assistedinject/4.0/guice-assistedinject-4.0-sources.jar"
             },
             {
                 "coord": "com.google.inject:guice:4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0.jar",
-                "directDependencies": [
-                    "aopalliance:aopalliance:1.0",
+                "dependencies": [
                     "com.google.guava:guava:18.0",
+                    "aopalliance:aopalliance:1.0",
                     "javax.inject:javax.inject:1"
                 ],
-                "dependencies": [
-                    "com.google.guava:guava:18.0",
+                "directDependencies": [
                     "aopalliance:aopalliance:1.0",
+                    "com.google.guava:guava:18.0",
                     "javax.inject:javax.inject:1"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0.jar",
+                "file": "v1/https/jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0.jar",
                     "https://maven.google.com/com/google/inject/guice/4.0/guice-4.0.jar",
                     "https://repo1.maven.org/maven2/com/google/inject/guice/4.0/guice-4.0.jar"
                 ],
-                "sha256": "b378ffc35e7f7125b3c5f3a461d4591ae1685e3c781392f0c854ed7b7581d6d2"
+                "sha256": "b378ffc35e7f7125b3c5f3a461d4591ae1685e3c781392f0c854ed7b7581d6d2",
+                "url": "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0.jar"
             },
             {
                 "coord": "com.google.inject:guice:jar:sources:4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0-sources.jar",
-                "directDependencies": [
-                    "aopalliance:aopalliance:jar:sources:1.0",
-                    "com.google.guava:guava:jar:sources:18.0",
-                    "javax.inject:javax.inject:jar:sources:1"
-                ],
                 "dependencies": [
                     "aopalliance:aopalliance:jar:sources:1.0",
                     "javax.inject:javax.inject:jar:sources:1",
                     "com.google.guava:guava:jar:sources:18.0"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0-sources.jar",
+                "directDependencies": [
+                    "aopalliance:aopalliance:jar:sources:1.0",
+                    "com.google.guava:guava:jar:sources:18.0",
+                    "javax.inject:javax.inject:jar:sources:1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0-sources.jar",
                     "https://maven.google.com/com/google/inject/guice/4.0/guice-4.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/inject/guice/4.0/guice-4.0-sources.jar"
                 ],
-                "sha256": "5ae16a56d478312ecee129b241a3df0fc9016b241bd4a0cbcd6b33f900a1eba6"
+                "sha256": "5ae16a56d478312ecee129b241a3df0fc9016b241bd4a0cbcd6b33f900a1eba6",
+                "url": "https://jcenter.bintray.com/com/google/inject/guice/4.0/guice-4.0-sources.jar"
             },
             {
                 "coord": "com.google.oauth-client:google-oauth-client:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar",
-                "directDependencies": [
-                    "com.google.code.findbugs:jsr305:3.0.2",
-                    "com.google.http-client:google-http-client:1.22.0"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
@@ -1185,21 +1181,21 @@
                     "org.apache.httpcomponents:httpclient:4.5.2",
                     "org.apache.httpcomponents:httpcore:4.4.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar",
+                "directDependencies": [
+                    "com.google.code.findbugs:jsr305:3.0.2",
+                    "com.google.http-client:google-http-client:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar",
                     "https://maven.google.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar",
                     "https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar"
                 ],
-                "sha256": "a4c56168b3e042105d68cf136e40e74f6e27f63ed0a948df966b332678e19022"
+                "sha256": "a4c56168b3e042105d68cf136e40e74f6e27f63ed0a948df966b332678e19022",
+                "url": "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0.jar"
             },
             {
                 "coord": "com.google.oauth-client:google-oauth-client:jar:sources:1.22.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar",
-                "directDependencies": [
-                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
-                    "com.google.http-client:google-http-client:jar:sources:1.22.0"
-                ],
                 "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "commons-logging:commons-logging:jar:sources:1.2",
@@ -1208,105 +1204,102 @@
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar",
+                "directDependencies": [
+                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
+                    "com.google.http-client:google-http-client:jar:sources:1.22.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar",
                     "https://maven.google.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar"
                 ],
-                "sha256": "f0e2896e486fb08bcaaf539569d5aa70f2f66528af54e0d1df1bd6493977dabb"
+                "sha256": "f0e2896e486fb08bcaaf539569d5aa70f2f66528af54e0d1df1bd6493977dabb",
+                "url": "https://jcenter.bintray.com/com/google/oauth-client/google-oauth-client/1.22.0/google-oauth-client-1.22.0-sources.jar"
             },
             {
                 "coord": "com.google.protobuf:protobuf-java:3.4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar",
                     "https://maven.google.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar",
                     "https://repo1.maven.org/maven2/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar"
                 ],
-                "sha256": "dce7e66b32456a1b1198da0caff3a8acb71548658391e798c79369241e6490a4"
+                "sha256": "dce7e66b32456a1b1198da0caff3a8acb71548658391e798c79369241e6490a4",
+                "url": "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0.jar"
             },
             {
                 "coord": "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
-                "file": "v1/https/jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar",
                     "https://maven.google.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar"
                 ],
-                "sha256": "07a55d5d34d2b47d2d1d9092be1dbf1b1d99fffcea19b7eafba508de8daae2cd"
+                "sha256": "07a55d5d34d2b47d2d1d9092be1dbf1b1d99fffcea19b7eafba508de8daae2cd",
+                "url": "https://jcenter.bintray.com/com/google/protobuf/protobuf-java/3.4.0/protobuf-java-3.4.0-sources.jar"
             },
             {
                 "coord": "com.googlecode.json-simple:json-simple:1.1",
-                "file": "v1/https/jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar",
                     "https://maven.google.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar",
                     "https://repo1.maven.org/maven2/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar"
                 ],
-                "sha256": "2d9484f4c649f708f47f9a479465fc729770ee65617dca3011836602264f6439"
+                "sha256": "2d9484f4c649f708f47f9a479465fc729770ee65617dca3011836602264f6439",
+                "url": "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1.jar"
             },
             {
                 "coord": "com.googlecode.json-simple:json-simple:jar:sources:1.1",
-                "file": "v1/https/jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar",
                     "https://maven.google.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar",
                     "https://repo1.maven.org/maven2/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar"
                 ],
-                "sha256": "883fa2afc741f7b1bf5ed2235bf8f5bf0de6dcdeb0457c11d41f2c1140417a53"
+                "sha256": "883fa2afc741f7b1bf5ed2235bf8f5bf0de6dcdeb0457c11d41f2c1140417a53",
+                "url": "https://jcenter.bintray.com/com/googlecode/json-simple/json-simple/1.1/json-simple-1.1-sources.jar"
             },
             {
                 "coord": "com.hashicorp.nomad:nomad-sdk:0.7.0",
-                "file": "v1/https/jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
+                    "commons-logging:commons-logging:1.2",
+                    "commons-codec:commons-codec:1.11",
                     "org.bouncycastle:bcpkix-jdk15on:1.61",
                     "org.apache.httpcomponents:httpclient:4.5.2",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
+                    "org.apache.httpcomponents:httpcore:4.4.4",
+                    "org.bouncycastle:bcprov-jdk15on:jar:1.61",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
-                    "commons-logging:commons-logging:1.2",
-                    "commons-codec:commons-codec:1.11",
                     "org.bouncycastle:bcpkix-jdk15on:1.61",
                     "org.apache.httpcomponents:httpclient:4.5.2",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
-                    "org.apache.httpcomponents:httpcore:4.4.4",
-                    "org.bouncycastle:bcprov-jdk15on:jar:1.61",
                     "com.fasterxml.jackson.core:jackson-core:2.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar",
+                "file": "v1/https/jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar",
                     "https://maven.google.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar",
                     "https://repo1.maven.org/maven2/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar"
                 ],
-                "sha256": "d04dda58d0242f87e66b333f5143d0f3aabb7e8ce1653c0ed1293a648cb18541"
+                "sha256": "d04dda58d0242f87e66b333f5143d0f3aabb7e8ce1653c0ed1293a648cb18541",
+                "url": "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0.jar"
             },
             {
                 "coord": "com.hashicorp.nomad:nomad-sdk:jar:sources:0.7.0",
-                "file": "v1/https/jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
-                    "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
-                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:jar:sources:1.2",
                     "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
@@ -1318,83 +1311,83 @@
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2",
                     "org.bouncycastle:bcprov-jdk15on:jar:sources:1.61"
                 ],
-                "url": "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
+                    "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
+                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar",
                     "https://maven.google.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar"
                 ],
-                "sha256": "d99fc9aca500dff6cfc41372c0e8178ad4b99a665fee4653282a2871b1ab001b"
+                "sha256": "d99fc9aca500dff6cfc41372c0e8178ad4b99a665fee4653282a2871b1ab001b",
+                "url": "https://jcenter.bintray.com/com/hashicorp/nomad/nomad-sdk/0.7.0/nomad-sdk-0.7.0-sources.jar"
             },
             {
                 "coord": "com.jayway.jsonpath:json-path:2.1.0",
-                "file": "v1/https/jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar",
-                "directDependencies": [
-                    "net.minidev:json-smart:2.2",
-                    "org.slf4j:slf4j-api:1.7.30"
-                ],
                 "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
                     "net.minidev:json-smart:2.2",
                     "net.minidev:accessors-smart:1.1",
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar",
+                "directDependencies": [
+                    "net.minidev:json-smart:2.2",
+                    "org.slf4j:slf4j-api:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar",
                     "https://maven.google.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar",
                     "https://repo1.maven.org/maven2/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar"
                 ],
-                "sha256": "9927c3e3472459e3c48f4a73224a7040dcb877cf96151fb3f19d6479dffccae9"
+                "sha256": "9927c3e3472459e3c48f4a73224a7040dcb877cf96151fb3f19d6479dffccae9",
+                "url": "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0.jar"
             },
             {
                 "coord": "com.jayway.jsonpath:json-path:jar:sources:2.1.0",
-                "file": "v1/https/jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar",
-                "directDependencies": [
-                    "net.minidev:json-smart:jar:sources:2.2",
-                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
-                ],
                 "dependencies": [
                     "net.minidev:accessors-smart:jar:sources:1.1",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "net.minidev:json-smart:jar:sources:2.2",
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar",
+                "directDependencies": [
+                    "net.minidev:json-smart:jar:sources:2.2",
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar",
                     "https://maven.google.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar"
                 ],
-                "sha256": "d2a1e6f45866f350d97bc9ef9e6d4dcc6e141f19573b09d1b292c6230e6808f8"
+                "sha256": "d2a1e6f45866f350d97bc9ef9e6d4dcc6e141f19573b09d1b292c6230e6808f8",
+                "url": "https://jcenter.bintray.com/com/jayway/jsonpath/json-path/2.1.0/json-path-2.1.0-sources.jar"
             },
             {
                 "coord": "com.jcraft:jsch:0.1.23",
-                "file": "v1/https/jcenter.bintray.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar",
-                "directDependencies": [],
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar",
+                "file": "v1/https/jcenter.bintray.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar",
                     "https://maven.google.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar",
                     "https://repo1.maven.org/maven2/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar"
                 ],
-                "sha256": "dd93757441e47aef6b6cd55f735821bae64eebaacc3aa808e951b3370e8e972a"
+                "sha256": "dd93757441e47aef6b6cd55f735821bae64eebaacc3aa808e951b3370e8e972a",
+                "url": "https://jcenter.bintray.com/com/jcraft/jsch/0.1.23/jsch-0.1.23.jar"
             },
             {
                 "coord": "com.microsoft.dhalion:dhalion:0.2.3",
-                "file": "v1/https/jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar",
-                "directDependencies": [
-                    "com.google.inject:guice:4.0",
-                    "com.google.inject.extensions:guice-assistedinject:4.0",
-                    "org.yaml:snakeyaml:1.15",
-                    "commons-cli:commons-cli:1.3.1",
-                    "tech.tablesaw:tablesaw-core:0.11.4"
-                ],
                 "dependencies": [
                     "com.google.inject:guice:4.0",
                     "com.google.inject.extensions:guice-assistedinject:4.0",
@@ -1417,24 +1410,24 @@
                     "com.opencsv:opencsv:4.1",
                     "commons-collections:commons-collections:3.2.1"
                 ],
-                "url": "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar",
+                "directDependencies": [
+                    "com.google.inject:guice:4.0",
+                    "com.google.inject.extensions:guice-assistedinject:4.0",
+                    "org.yaml:snakeyaml:1.15",
+                    "commons-cli:commons-cli:1.3.1",
+                    "tech.tablesaw:tablesaw-core:0.11.4"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar",
                     "https://maven.google.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar",
                     "https://repo1.maven.org/maven2/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar"
                 ],
-                "sha256": "89c379170708c0304582cdd55fc887980c7f8e8427391d18eb920a1e7fe4d7d1"
+                "sha256": "89c379170708c0304582cdd55fc887980c7f8e8427391d18eb920a1e7fe4d7d1",
+                "url": "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3.jar"
             },
             {
                 "coord": "com.microsoft.dhalion:dhalion:jar:sources:0.2.3",
-                "file": "v1/https/jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar",
-                "directDependencies": [
-                    "org.yaml:snakeyaml:jar:sources:1.15",
-                    "com.google.inject:guice:jar:sources:4.0",
-                    "tech.tablesaw:tablesaw-core:jar:sources:0.11.4",
-                    "com.google.inject.extensions:guice-assistedinject:jar:sources:4.0",
-                    "commons-cli:commons-cli:jar:sources:1.3.1"
-                ],
                 "dependencies": [
                     "com.opencsv:opencsv:jar:sources:4.1",
                     "org.jsoup:jsoup:jar:sources:1.11.2",
@@ -1457,22 +1450,24 @@
                     "com.google.inject.extensions:guice-assistedinject:jar:sources:4.0",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar",
+                "directDependencies": [
+                    "org.yaml:snakeyaml:jar:sources:1.15",
+                    "com.google.inject:guice:jar:sources:4.0",
+                    "tech.tablesaw:tablesaw-core:jar:sources:0.11.4",
+                    "com.google.inject.extensions:guice-assistedinject:jar:sources:4.0",
+                    "commons-cli:commons-cli:jar:sources:1.3.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar",
                     "https://maven.google.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar",
                     "https://repo1.maven.org/maven2/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar"
                 ],
-                "sha256": "8c062f77ee30aa866695ddc1647939f1ba3a1394cd4da03f05a26dfbe50dcb95"
+                "sha256": "8c062f77ee30aa866695ddc1647939f1ba3a1394cd4da03f05a26dfbe50dcb95",
+                "url": "https://jcenter.bintray.com/com/microsoft/dhalion/dhalion/0.2.3/dhalion-0.2.3-sources.jar"
             },
             {
                 "coord": "com.opencsv:opencsv:4.1",
-                "file": "v1/https/jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar",
-                "directDependencies": [
-                    "commons-beanutils:commons-beanutils:1.9.3",
-                    "org.apache.commons:commons-lang3:3.7",
-                    "org.apache.commons:commons-text:1.1"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "org.apache.commons:commons-lang3:3.7",
@@ -1480,22 +1475,22 @@
                     "commons-beanutils:commons-beanutils:1.9.3",
                     "commons-collections:commons-collections:3.2.1"
                 ],
-                "url": "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar",
+                "directDependencies": [
+                    "commons-beanutils:commons-beanutils:1.9.3",
+                    "org.apache.commons:commons-lang3:3.7",
+                    "org.apache.commons:commons-text:1.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar",
                     "https://maven.google.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar",
                     "https://repo1.maven.org/maven2/com/opencsv/opencsv/4.1/opencsv-4.1.jar"
                 ],
-                "sha256": "68283bf170ce9ca7eb70a390fbb590259455a7f7651b96ce2864eb2dde2bfbbd"
+                "sha256": "68283bf170ce9ca7eb70a390fbb590259455a7f7651b96ce2864eb2dde2bfbbd",
+                "url": "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1.jar"
             },
             {
                 "coord": "com.opencsv:opencsv:jar:sources:4.1",
-                "file": "v1/https/jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar",
-                "directDependencies": [
-                    "commons-beanutils:commons-beanutils:jar:sources:1.9.3",
-                    "org.apache.commons:commons-lang3:jar:sources:3.7",
-                    "org.apache.commons:commons-text:jar:sources:1.1"
-                ],
                 "dependencies": [
                     "org.apache.commons:commons-text:jar:sources:1.1",
                     "commons-logging:commons-logging:jar:sources:1.2",
@@ -1503,1168 +1498,1171 @@
                     "commons-collections:commons-collections:jar:sources:3.2.1",
                     "commons-beanutils:commons-beanutils:jar:sources:1.9.3"
                 ],
-                "url": "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar",
+                "directDependencies": [
+                    "commons-beanutils:commons-beanutils:jar:sources:1.9.3",
+                    "org.apache.commons:commons-lang3:jar:sources:3.7",
+                    "org.apache.commons:commons-text:jar:sources:1.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar",
                     "https://maven.google.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar",
                     "https://repo1.maven.org/maven2/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar"
                 ],
-                "sha256": "40d9c01e92831e004650380ba19e881c2faa5ecd96076986ea52b3ecf830b2f6"
+                "sha256": "40d9c01e92831e004650380ba19e881c2faa5ecd96076986ea52b3ecf830b2f6",
+                "url": "https://jcenter.bintray.com/com/opencsv/opencsv/4.1/opencsv-4.1-sources.jar"
             },
             {
                 "coord": "com.puppycrawl.tools:checkstyle:6.17",
-                "file": "v1/https/jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "commons-logging:commons-logging:1.2",
                     "org.antlr:antlr4-runtime:4.5.2-1",
                     "antlr:antlr:2.7.7",
                     "commons-cli:commons-cli:1.3.1",
                     "com.google.guava:guava:18.0",
-                    "commons-beanutils:commons-beanutils:1.9.3"
+                    "commons-beanutils:commons-beanutils:1.9.3",
+                    "commons-collections:commons-collections:3.2.1"
                 ],
-                "dependencies": [
-                    "commons-logging:commons-logging:1.2",
+                "directDependencies": [
                     "org.antlr:antlr4-runtime:4.5.2-1",
                     "antlr:antlr:2.7.7",
                     "commons-cli:commons-cli:1.3.1",
                     "com.google.guava:guava:18.0",
-                    "commons-beanutils:commons-beanutils:1.9.3",
-                    "commons-collections:commons-collections:3.2.1"
+                    "commons-beanutils:commons-beanutils:1.9.3"
                 ],
-                "url": "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar",
+                "file": "v1/https/jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar",
                     "https://maven.google.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar",
                     "https://repo1.maven.org/maven2/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar"
                 ],
-                "sha256": "61a8b52d03a5b163d0983cdc4b03396a92ea7f8dc8c007dda30f4db673e9e60c"
+                "sha256": "61a8b52d03a5b163d0983cdc4b03396a92ea7f8dc8c007dda30f4db673e9e60c",
+                "url": "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17.jar"
             },
             {
                 "coord": "com.puppycrawl.tools:checkstyle:jar:sources:6.17",
-                "file": "v1/https/jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "commons-logging:commons-logging:jar:sources:1.2",
                     "com.google.guava:guava:jar:sources:18.0",
+                    "commons-collections:commons-collections:jar:sources:3.2.1",
                     "org.antlr:antlr4-runtime:jar:sources:4.5.2-1",
                     "antlr:antlr:jar:sources:2.7.7",
                     "commons-beanutils:commons-beanutils:jar:sources:1.9.3",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "dependencies": [
-                    "commons-logging:commons-logging:jar:sources:1.2",
+                "directDependencies": [
                     "com.google.guava:guava:jar:sources:18.0",
-                    "commons-collections:commons-collections:jar:sources:3.2.1",
                     "org.antlr:antlr4-runtime:jar:sources:4.5.2-1",
                     "antlr:antlr:jar:sources:2.7.7",
                     "commons-beanutils:commons-beanutils:jar:sources:1.9.3",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar",
                     "https://maven.google.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar",
                     "https://repo1.maven.org/maven2/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar"
                 ],
-                "sha256": "8cea4d0cab18e19502f91d07bb572f9776f86539445117dc790aa86774b399e7"
+                "sha256": "8cea4d0cab18e19502f91d07bb572f9776f86539445117dc790aa86774b399e7",
+                "url": "https://jcenter.bintray.com/com/puppycrawl/tools/checkstyle/6.17/checkstyle-6.17-sources.jar"
             },
             {
                 "coord": "com.squareup.okhttp3:logging-interceptor:3.14.3",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar",
-                "directDependencies": [
-                    "com.squareup.okhttp3:okhttp:3.14.3"
-                ],
                 "dependencies": [
                     "com.squareup.okhttp3:okhttp:3.14.3",
                     "com.squareup.okio:okio:1.17.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar",
+                "directDependencies": [
+                    "com.squareup.okhttp3:okhttp:3.14.3"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar",
                     "https://maven.google.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar"
                 ],
-                "sha256": "4b76b6c85af32c8b4fab2ffd1844e6d11530bc682fa65bd9490c46cfff0d08fd"
+                "sha256": "4b76b6c85af32c8b4fab2ffd1844e6d11530bc682fa65bd9490c46cfff0d08fd",
+                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3.jar"
             },
             {
                 "coord": "com.squareup.okhttp3:logging-interceptor:jar:sources:3.14.3",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar",
-                "directDependencies": [
-                    "com.squareup.okhttp3:okhttp:jar:sources:3.14.3"
-                ],
                 "dependencies": [
                     "com.squareup.okhttp3:okhttp:jar:sources:3.14.3",
                     "com.squareup.okio:okio:jar:sources:1.17.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar",
+                "directDependencies": [
+                    "com.squareup.okhttp3:okhttp:jar:sources:3.14.3"
+                ],
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar",
                     "https://maven.google.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar"
                 ],
-                "sha256": "2b3bcac909ece65b6b8744cecc8a6c45195655fcb3007ce2de58e326ef02fe69"
+                "sha256": "2b3bcac909ece65b6b8744cecc8a6c45195655fcb3007ce2de58e326ef02fe69",
+                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/logging-interceptor/3.14.3/logging-interceptor-3.14.3-sources.jar"
             },
             {
                 "coord": "com.squareup.okhttp3:okhttp:3.14.3",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.squareup.okio:okio:1.17.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.squareup.okio:okio:1.17.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar",
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar",
                     "https://maven.google.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar"
                 ],
-                "sha256": "f84f7680cb8ce303d5e7425520ac8f14e044877d3f34cb63545de79ea974501a"
+                "sha256": "f84f7680cb8ce303d5e7425520ac8f14e044877d3f34cb63545de79ea974501a",
+                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3.jar"
             },
             {
                 "coord": "com.squareup.okhttp3:okhttp:jar:sources:3.14.3",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.squareup.okio:okio:jar:sources:1.17.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.squareup.okio:okio:jar:sources:1.17.2"
                 ],
-                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar",
                     "https://maven.google.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar"
                 ],
-                "sha256": "de348980c90a1b083081f8b0d4286150e859a858219ac97b5805ff4650d52ca1"
+                "sha256": "de348980c90a1b083081f8b0d4286150e859a858219ac97b5805ff4650d52ca1",
+                "url": "https://jcenter.bintray.com/com/squareup/okhttp3/okhttp/3.14.3/okhttp-3.14.3-sources.jar"
             },
             {
                 "coord": "com.squareup.okio:okio:1.17.2",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar",
                     "https://maven.google.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar"
                 ],
-                "sha256": "f80ce42d2ffac47ad4c47e1d6f980d604d247ceb1a886705cf4581ab0c9fe2b8"
+                "sha256": "f80ce42d2ffac47ad4c47e1d6f980d604d247ceb1a886705cf4581ab0c9fe2b8",
+                "url": "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2.jar"
             },
             {
                 "coord": "com.squareup.okio:okio:jar:sources:1.17.2",
-                "file": "v1/https/jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar",
                     "https://maven.google.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar",
                     "https://repo1.maven.org/maven2/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar"
                 ],
-                "sha256": "f96949004bdc5951c27256b96c33667d3dbd5369c467a8fa0d4157dba98b4252"
+                "sha256": "f96949004bdc5951c27256b96c33667d3dbd5369c467a8fa0d4157dba98b4252",
+                "url": "https://jcenter.bintray.com/com/squareup/okio/okio/1.17.2/okio-1.17.2-sources.jar"
             },
             {
                 "coord": "com.thoughtworks.paranamer:paranamer:2.3",
-                "file": "v1/https/jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar",
                     "https://maven.google.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar",
                     "https://repo1.maven.org/maven2/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar"
                 ],
-                "sha256": "e93f50ae4d0de11080677f44ab268691266fed2b3ff7bc6fd97636febae7d8fe"
+                "sha256": "e93f50ae4d0de11080677f44ab268691266fed2b3ff7bc6fd97636febae7d8fe",
+                "url": "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar"
             },
             {
                 "coord": "com.thoughtworks.paranamer:paranamer:jar:sources:2.3",
-                "file": "v1/https/jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar",
                     "https://maven.google.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar",
                     "https://repo1.maven.org/maven2/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar"
                 ],
-                "sha256": "5691dc48d5f9a37303c7fd0d13fc3984edb4700030a498211924e5fb44d54ce2"
+                "sha256": "5691dc48d5f9a37303c7fd0d13fc3984edb4700030a498211924e5fb44d54ce2",
+                "url": "https://jcenter.bintray.com/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3-sources.jar"
             },
             {
                 "coord": "com.typesafe.netty:netty-reactive-streams:1.0.8",
-                "file": "v1/https/jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.reactivestreams:reactive-streams:1.0.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.reactivestreams:reactive-streams:1.0.0"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar",
+                "file": "v1/https/jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar",
                     "https://maven.google.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar",
                     "https://repo1.maven.org/maven2/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar"
                 ],
-                "sha256": "a368c024079971086d83c89ff7fbd6a028e6a8185faafd6ab8339e760f0f7aa2"
+                "sha256": "a368c024079971086d83c89ff7fbd6a028e6a8185faafd6ab8339e760f0f7aa2",
+                "url": "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8.jar"
             },
             {
                 "coord": "com.typesafe.netty:netty-reactive-streams:jar:sources:1.0.8",
-                "file": "v1/https/jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.reactivestreams:reactive-streams:jar:sources:1.0.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.reactivestreams:reactive-streams:jar:sources:1.0.0"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar",
                     "https://maven.google.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar"
                 ],
-                "sha256": "640fd613fa17a20203d03a8d58b98a8069aef8b4930a821166b1a9857e12e3d9"
+                "sha256": "640fd613fa17a20203d03a8d58b98a8069aef8b4930a821166b1a9857e12e3d9",
+                "url": "https://jcenter.bintray.com/com/typesafe/netty/netty-reactive-streams/1.0.8/netty-reactive-streams-1.0.8-sources.jar"
             },
             {
                 "coord": "com.yahoo.datasketches:sketches-core:0.6.0",
-                "file": "v1/https/jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar",
                     "https://maven.google.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar",
                     "https://repo1.maven.org/maven2/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar"
                 ],
-                "sha256": "1c24ff2aeb54934d7f45d2c126e40cffe675b4357882bb9134c3c0190fcd51c3"
+                "sha256": "1c24ff2aeb54934d7f45d2c126e40cffe675b4357882bb9134c3c0190fcd51c3",
+                "url": "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0.jar"
             },
             {
                 "coord": "com.yahoo.datasketches:sketches-core:jar:sources:0.6.0",
-                "file": "v1/https/jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar",
                     "https://maven.google.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar",
                     "https://repo1.maven.org/maven2/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar"
                 ],
-                "sha256": "0778d928d72834ff992502cfa6f6c14d594baa95bf1259185c4e08f3bf425c26"
+                "sha256": "0778d928d72834ff992502cfa6f6c14d594baa95bf1259185c4e08f3bf425c26",
+                "url": "https://jcenter.bintray.com/com/yahoo/datasketches/sketches-core/0.6.0/sketches-core-0.6.0-sources.jar"
             },
             {
                 "coord": "commons-beanutils:commons-beanutils:1.9.3",
-                "file": "v1/https/jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-collections:commons-collections:3.2.1",
                     "commons-logging:commons-logging:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-collections:commons-collections:3.2.1",
                     "commons-logging:commons-logging:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar",
                     "https://maven.google.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar",
                     "https://repo1.maven.org/maven2/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar"
                 ],
-                "sha256": "c058e39c7c64203d3a448f3adb588cb03d6378ed808485618f26e137f29dae73"
+                "sha256": "c058e39c7c64203d3a448f3adb588cb03d6378ed808485618f26e137f29dae73",
+                "url": "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar"
             },
             {
                 "coord": "commons-beanutils:commons-beanutils:jar:sources:1.9.3",
-                "file": "v1/https/jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-collections:commons-collections:jar:sources:3.2.1",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-collections:commons-collections:jar:sources:3.2.1",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar",
                     "https://maven.google.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar",
                     "https://repo1.maven.org/maven2/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar"
                 ],
-                "sha256": "3315f25f3793c1bb2577b2d956f58f852c7386c73aff4dea450e419a80b00a41"
+                "sha256": "3315f25f3793c1bb2577b2d956f58f852c7386c73aff4dea450e419a80b00a41",
+                "url": "https://jcenter.bintray.com/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3-sources.jar"
             },
             {
                 "coord": "commons-cli:commons-cli:1.3.1",
-                "file": "v1/https/jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar",
                     "https://maven.google.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar",
                     "https://repo1.maven.org/maven2/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar"
                 ],
-                "sha256": "3a2f057041aa6a8813f5b59b695f726c5e85014a703d208d7e1689098e92d8c0"
+                "sha256": "3a2f057041aa6a8813f5b59b695f726c5e85014a703d208d7e1689098e92d8c0",
+                "url": "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1.jar"
             },
             {
                 "coord": "commons-cli:commons-cli:jar:sources:1.3.1",
-                "file": "v1/https/jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar",
                     "https://maven.google.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar",
                     "https://repo1.maven.org/maven2/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar"
                 ],
-                "sha256": "b440fdddb4c82001c6c1ed4d89d5cc1eee7dd21049b00ae3722f14ffd4bdf6cc"
+                "sha256": "b440fdddb4c82001c6c1ed4d89d5cc1eee7dd21049b00ae3722f14ffd4bdf6cc",
+                "url": "https://jcenter.bintray.com/commons-cli/commons-cli/1.3.1/commons-cli-1.3.1-sources.jar"
             },
             {
                 "coord": "commons-codec:commons-codec:1.11",
-                "file": "v1/https/jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar",
                     "https://maven.google.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar",
                     "https://repo1.maven.org/maven2/commons-codec/commons-codec/1.11/commons-codec-1.11.jar"
                 ],
-                "sha256": "e599d5318e97aa48f42136a2927e6dfa4e8881dff0e6c8e3109ddbbff51d7b7d"
+                "sha256": "e599d5318e97aa48f42136a2927e6dfa4e8881dff0e6c8e3109ddbbff51d7b7d",
+                "url": "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar"
             },
             {
                 "coord": "commons-codec:commons-codec:jar:sources:1.11",
-                "file": "v1/https/jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar",
                     "https://maven.google.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar",
                     "https://repo1.maven.org/maven2/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar"
                 ],
-                "sha256": "901cb5d1f7c2877017c95d3c5efd5a497738d0162ef72cdf58e9cb13f50b2e9c"
+                "sha256": "901cb5d1f7c2877017c95d3c5efd5a497738d0162ef72cdf58e9cb13f50b2e9c",
+                "url": "https://jcenter.bintray.com/commons-codec/commons-codec/1.11/commons-codec-1.11-sources.jar"
             },
             {
                 "coord": "commons-collections:commons-collections:3.2.1",
-                "file": "v1/https/jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar",
                     "https://maven.google.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar",
                     "https://repo1.maven.org/maven2/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar"
                 ],
-                "sha256": "87363a4c94eaabeefd8b930cb059f66b64c9f7d632862f23de3012da7660047b"
+                "sha256": "87363a4c94eaabeefd8b930cb059f66b64c9f7d632862f23de3012da7660047b",
+                "url": "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar"
             },
             {
                 "coord": "commons-collections:commons-collections:jar:sources:3.2.1",
-                "file": "v1/https/jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar",
                     "https://maven.google.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar",
                     "https://repo1.maven.org/maven2/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar"
                 ],
-                "sha256": "5bd4cfb6db784c74d344fd6c39801763e0bf5d553ae1e06d5357673f63883cec"
+                "sha256": "5bd4cfb6db784c74d344fd6c39801763e0bf5d553ae1e06d5357673f63883cec",
+                "url": "https://jcenter.bintray.com/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1-sources.jar"
             },
             {
                 "coord": "commons-configuration:commons-configuration:1.10",
-                "file": "v1/https/jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar",
-                "directDependencies": [
-                    "commons-lang:commons-lang:2.6",
-                    "commons-logging:commons-logging:1.2"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "commons-lang:commons-lang:2.6"
                 ],
-                "url": "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar",
+                "directDependencies": [
+                    "commons-lang:commons-lang:2.6",
+                    "commons-logging:commons-logging:1.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar",
                     "https://maven.google.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar",
                     "https://repo1.maven.org/maven2/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar"
                 ],
-                "sha256": "95d4e6711e88ce78992c82c25bc03c8df9ecf5a357f0de0bec72a26db3399374"
+                "sha256": "95d4e6711e88ce78992c82c25bc03c8df9ecf5a357f0de0bec72a26db3399374",
+                "url": "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10.jar"
             },
             {
                 "coord": "commons-configuration:commons-configuration:jar:sources:1.10",
-                "file": "v1/https/jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-lang:commons-lang:jar:sources:2.6",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-lang:commons-lang:jar:sources:2.6",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar",
                     "https://maven.google.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar",
                     "https://repo1.maven.org/maven2/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar"
                 ],
-                "sha256": "0dde29a828f51e142d8392b20e4d69edd7d55ba5ea05a288e4ddc2222ecf5ced"
+                "sha256": "0dde29a828f51e142d8392b20e4d69edd7d55ba5ea05a288e4ddc2222ecf5ced",
+                "url": "https://jcenter.bintray.com/commons-configuration/commons-configuration/1.10/commons-configuration-1.10-sources.jar"
             },
             {
                 "coord": "commons-el:commons-el:1.0",
-                "file": "v1/https/jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-logging:commons-logging:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-logging:commons-logging:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0.jar",
                     "https://maven.google.com/commons-el/commons-el/1.0/commons-el-1.0.jar",
                     "https://repo1.maven.org/maven2/commons-el/commons-el/1.0/commons-el-1.0.jar"
                 ],
-                "sha256": "0d67550ec0022b653453c759f063a643c2fe64bc48faa8b25f95a220e2a282e2"
+                "sha256": "0d67550ec0022b653453c759f063a643c2fe64bc48faa8b25f95a220e2a282e2",
+                "url": "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0.jar"
             },
             {
                 "coord": "commons-el:commons-el:jar:sources:1.0",
-                "file": "v1/https/jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar",
                     "https://maven.google.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar",
                     "https://repo1.maven.org/maven2/commons-el/commons-el/1.0/commons-el-1.0-sources.jar"
                 ],
-                "sha256": "2c6617fa865c60894c0f28e2f621a322eec518fd201f9206ec7a07fb0c3421ca"
+                "sha256": "2c6617fa865c60894c0f28e2f621a322eec518fd201f9206ec7a07fb0c3421ca",
+                "url": "https://jcenter.bintray.com/commons-el/commons-el/1.0/commons-el-1.0-sources.jar"
             },
             {
                 "coord": "commons-httpclient:commons-httpclient:3.1",
-                "file": "v1/https/jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:1.11",
-                    "commons-logging:commons-logging:1.2"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "commons-codec:commons-codec:1.11"
                 ],
-                "url": "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar",
+                "directDependencies": [
+                    "commons-codec:commons-codec:1.11",
+                    "commons-logging:commons-logging:1.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar",
                     "https://maven.google.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar",
                     "https://repo1.maven.org/maven2/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar"
                 ],
-                "sha256": "dbd4953d013e10e7c1cc3701a3e6ccd8c950c892f08d804fabfac21705930443"
+                "sha256": "dbd4953d013e10e7c1cc3701a3e6ccd8c950c892f08d804fabfac21705930443",
+                "url": "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar"
             },
             {
                 "coord": "commons-httpclient:commons-httpclient:jar:sources:3.1",
-                "file": "v1/https/jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-codec:commons-codec:jar:sources:1.11",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-codec:commons-codec:jar:sources:1.11",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar",
                     "https://maven.google.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar",
                     "https://repo1.maven.org/maven2/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar"
                 ],
-                "sha256": "d1664b998624bf8f7544aaf4d0145277901842ef91b291c13133f01cb6adcc15"
+                "sha256": "d1664b998624bf8f7544aaf4d0145277901842ef91b291c13133f01cb6adcc15",
+                "url": "https://jcenter.bintray.com/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1-sources.jar"
             },
             {
                 "coord": "commons-io:commons-io:2.4",
-                "file": "v1/https/jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4.jar",
                     "https://maven.google.com/commons-io/commons-io/2.4/commons-io-2.4.jar",
                     "https://repo1.maven.org/maven2/commons-io/commons-io/2.4/commons-io-2.4.jar"
                 ],
-                "sha256": "cc6a41dc3eaacc9e440a6bd0d2890b20d36b4ee408fe2d67122f328bb6e01581"
+                "sha256": "cc6a41dc3eaacc9e440a6bd0d2890b20d36b4ee408fe2d67122f328bb6e01581",
+                "url": "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4.jar"
             },
             {
                 "coord": "commons-io:commons-io:jar:sources:2.4",
-                "file": "v1/https/jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar",
                     "https://maven.google.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar",
                     "https://repo1.maven.org/maven2/commons-io/commons-io/2.4/commons-io-2.4-sources.jar"
                 ],
-                "sha256": "d4635b348bbbf3f166d972b052bc4cac5b326c133beed7b8a1cab7ea22b61e01"
+                "sha256": "d4635b348bbbf3f166d972b052bc4cac5b326c133beed7b8a1cab7ea22b61e01",
+                "url": "https://jcenter.bintray.com/commons-io/commons-io/2.4/commons-io-2.4-sources.jar"
             },
             {
                 "coord": "commons-lang:commons-lang:2.6",
-                "file": "v1/https/jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar",
                     "https://maven.google.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar",
                     "https://repo1.maven.org/maven2/commons-lang/commons-lang/2.6/commons-lang-2.6.jar"
                 ],
-                "sha256": "50f11b09f877c294d56f24463f47d28f929cf5044f648661c0f0cfbae9a2f49c"
+                "sha256": "50f11b09f877c294d56f24463f47d28f929cf5044f648661c0f0cfbae9a2f49c",
+                "url": "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6.jar"
             },
             {
                 "coord": "commons-lang:commons-lang:jar:sources:2.6",
-                "file": "v1/https/jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar",
                     "https://maven.google.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar",
                     "https://repo1.maven.org/maven2/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar"
                 ],
-                "sha256": "66c2760945cec226f26286ddf3f6ffe38544c4a69aade89700a9a689c9b92380"
+                "sha256": "66c2760945cec226f26286ddf3f6ffe38544c4a69aade89700a9a689c9b92380",
+                "url": "https://jcenter.bintray.com/commons-lang/commons-lang/2.6/commons-lang-2.6-sources.jar"
             },
             {
                 "coord": "commons-logging:commons-logging-api:1.1",
-                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar",
                     "https://maven.google.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar",
                     "https://repo1.maven.org/maven2/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar"
                 ],
-                "sha256": "33a4dd47bb4764e4eb3692d86386d17a0d9827f4f4bb0f70121efab6bc03ba35"
+                "sha256": "33a4dd47bb4764e4eb3692d86386d17a0d9827f4f4bb0f70121efab6bc03ba35",
+                "url": "https://jcenter.bintray.com/commons-logging/commons-logging-api/1.1/commons-logging-api-1.1.jar"
             },
             {
                 "coord": "commons-logging:commons-logging:1.2",
-                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar",
                     "https://maven.google.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar",
                     "https://repo1.maven.org/maven2/commons-logging/commons-logging/1.2/commons-logging-1.2.jar"
                 ],
-                "sha256": "daddea1ea0be0f56978ab3006b8ac92834afeefbd9b7e4e6316fca57df0fa636"
+                "sha256": "daddea1ea0be0f56978ab3006b8ac92834afeefbd9b7e4e6316fca57df0fa636",
+                "url": "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar"
             },
             {
                 "coord": "commons-logging:commons-logging:jar:sources:1.2",
-                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar",
                     "https://maven.google.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar",
                     "https://repo1.maven.org/maven2/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar"
                 ],
-                "sha256": "44347acfe5860461728e9cb33251e97345be36f8a0dfd5c5130c172559455f41"
+                "sha256": "44347acfe5860461728e9cb33251e97345be36f8a0dfd5c5130c172559455f41",
+                "url": "https://jcenter.bintray.com/commons-logging/commons-logging/1.2/commons-logging-1.2-sources.jar"
             },
             {
                 "coord": "commons-net:commons-net:1.4.1",
-                "file": "v1/https/jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "oro:oro:2.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "oro:oro:2.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar",
                     "https://maven.google.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar",
                     "https://repo1.maven.org/maven2/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar"
                 ],
-                "sha256": "05a3611dedf90d0ab3e8ed83dec4ee49200148c09425437eb9348562fde7d83c"
+                "sha256": "05a3611dedf90d0ab3e8ed83dec4ee49200148c09425437eb9348562fde7d83c",
+                "url": "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar"
             },
             {
                 "coord": "commons-net:commons-net:jar:sources:1.4.1",
-                "file": "v1/https/jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "oro:oro:jar:sources:2.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "oro:oro:jar:sources:2.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar",
                     "https://maven.google.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar",
                     "https://repo1.maven.org/maven2/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar"
                 ],
-                "sha256": "83caad6228a66b7f3158c676b76817d0177ed7980159cec15dbeabd9d5a2d1d6"
+                "sha256": "83caad6228a66b7f3158c676b76817d0177ed7980159cec15dbeabd9d5a2d1d6",
+                "url": "https://jcenter.bintray.com/commons-net/commons-net/1.4.1/commons-net-1.4.1-sources.jar"
             },
             {
                 "coord": "doxia:doxia-sink-api:1.0-alpha-4",
-                "file": "v1/https/jcenter.bintray.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar",
-                "directDependencies": [],
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "plexus:plexus-container-default",
                     "plexus:plexus-utils"
                 ],
-                "url": "https://jcenter.bintray.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar",
+                "file": "v1/https/jcenter.bintray.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar",
                     "https://maven.google.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar",
                     "https://repo1.maven.org/maven2/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar"
                 ],
-                "sha256": "f609a721b6e6f7d24352b9b12d0853d832a247b1fab189cfd641347c5f5c522c"
+                "sha256": "f609a721b6e6f7d24352b9b12d0853d832a247b1fab189cfd641347c5f5c522c",
+                "url": "https://jcenter.bintray.com/doxia/doxia-sink-api/1.0-alpha-4/doxia-sink-api-1.0-alpha-4.jar"
             },
             {
                 "coord": "hsqldb:hsqldb:1.8.0.10",
-                "file": "v1/https/jcenter.bintray.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar",
                     "https://maven.google.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar",
                     "https://repo1.maven.org/maven2/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar"
                 ],
-                "sha256": "b04b3b3ac295d497c87230eeb4f888327a5a15b9c3c1567db202a51d83ac9e41"
+                "sha256": "b04b3b3ac295d497c87230eeb4f888327a5a15b9c3c1567db202a51d83ac9e41",
+                "url": "https://jcenter.bintray.com/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar"
             },
             {
                 "coord": "io.gsonfire:gson-fire:1.8.3",
-                "file": "v1/https/jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.google.code.gson:gson:2.8.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.code.gson:gson:2.8.2"
                 ],
-                "url": "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar",
+                "file": "v1/https/jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar",
                     "https://maven.google.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar",
                     "https://repo1.maven.org/maven2/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar"
                 ],
-                "sha256": "fabe45ed428005d971ad7b27f5c7a618790e1314c62a7d0da5f1090697da4f37"
+                "sha256": "fabe45ed428005d971ad7b27f5c7a618790e1314c62a7d0da5f1090697da4f37",
+                "url": "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3.jar"
             },
             {
                 "coord": "io.gsonfire:gson-fire:jar:sources:1.8.3",
-                "file": "v1/https/jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.google.code.gson:gson:jar:sources:2.8.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.code.gson:gson:jar:sources:2.8.2"
                 ],
-                "url": "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar",
                     "https://maven.google.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar",
                     "https://repo1.maven.org/maven2/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar"
                 ],
-                "sha256": "c6c5e785a30b6a496d5037910b77177494ea986fac24d8c43232547a37359128"
+                "sha256": "c6c5e785a30b6a496d5037910b77177494ea986fac24d8c43232547a37359128",
+                "url": "https://jcenter.bintray.com/io/gsonfire/gson-fire/1.8.3/gson-fire-1.8.3-sources.jar"
             },
             {
-                "coord": "io.kubernetes:client-java-api:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0.jar",
-                "directDependencies": [
+                "coord": "io.kubernetes:client-java-api:8.0.0",
+                "dependencies": [
                     "io.swagger:swagger-annotations:1.5.22",
                     "joda-time:joda-time:2.9.3",
                     "com.squareup.okhttp3:okhttp:3.14.3",
                     "com.google.code.findbugs:jsr305:3.0.2",
+                    "io.sundr:sundr-core:0.21.0",
                     "org.apache.commons:commons-lang3:3.7",
+                    "io.sundr:builder-annotations:0.21.0",
+                    "io.sundr:sundr-codegen:0.21.0",
                     "com.squareup.okhttp3:logging-interceptor:3.14.3",
                     "org.joda:joda-convert:1.2",
+                    "io.sundr:resourcecify-annotations:0.21.0",
                     "javax.annotation:javax.annotation-api:1.3.2",
-                    "io.sundr:builder-annotations:0.19.2",
                     "com.google.code.gson:gson:2.8.2",
+                    "com.squareup.okio:okio:1.17.2",
                     "io.gsonfire:gson-fire:1.8.3"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.swagger:swagger-annotations:1.5.22",
                     "joda-time:joda-time:2.9.3",
                     "com.squareup.okhttp3:okhttp:3.14.3",
                     "com.google.code.findbugs:jsr305:3.0.2",
-                    "io.sundr:resourcecify-annotations:0.19.2",
-                    "io.sundr:sundr-core:0.19.2",
                     "org.apache.commons:commons-lang3:3.7",
+                    "io.sundr:builder-annotations:0.21.0",
                     "com.squareup.okhttp3:logging-interceptor:3.14.3",
-                    "io.sundr:sundr-codegen:0.19.2",
                     "org.joda:joda-convert:1.2",
                     "javax.annotation:javax.annotation-api:1.3.2",
-                    "io.sundr:builder-annotations:0.19.2",
                     "com.google.code.gson:gson:2.8.2",
-                    "com.squareup.okio:okio:1.17.2",
                     "io.gsonfire:gson-fire:1.8.3"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0.jar",
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0.jar",
-                    "https://maven.google.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0.jar",
+                    "https://maven.google.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0.jar"
                 ],
-                "sha256": "8affdb062e736f0aab64e5c1f0ba738d33bd04082f75437262aa024181c693d8"
+                "sha256": "9f182ac8aebb00e800e0688a66fa7c21df0499a5a976d22e68c9af8d9a0b1c9c",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0.jar"
             },
             {
-                "coord": "io.kubernetes:client-java-api:jar:sources:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0-sources.jar",
-                "directDependencies": [
+                "coord": "io.kubernetes:client-java-api:jar:sources:8.0.0",
+                "dependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "joda-time:joda-time:jar:sources:2.9.3",
-                    "io.sundr:builder-annotations:jar:sources:0.19.2",
+                    "io.sundr:resourcecify-annotations:jar:sources:0.21.0",
+                    "io.sundr:sundr-codegen:jar:sources:0.21.0",
                     "org.apache.commons:commons-lang3:jar:sources:3.7",
                     "com.google.code.gson:gson:jar:sources:2.8.2",
                     "org.joda:joda-convert:jar:sources:1.2",
                     "com.squareup.okhttp3:logging-interceptor:jar:sources:3.14.3",
                     "io.gsonfire:gson-fire:jar:sources:1.8.3",
+                    "io.sundr:sundr-core:jar:sources:0.21.0",
+                    "io.sundr:builder-annotations:jar:sources:0.21.0",
                     "io.swagger:swagger-annotations:jar:sources:1.5.22",
                     "com.squareup.okhttp3:okhttp:jar:sources:3.14.3",
-                    "javax.annotation:javax.annotation-api:jar:sources:1.3.2"
+                    "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
+                    "com.squareup.okio:okio:jar:sources:1.17.2"
                 ],
-                "dependencies": [
-                    "io.sundr:resourcecify-annotations:jar:sources:0.19.2",
+                "directDependencies": [
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "joda-time:joda-time:jar:sources:2.9.3",
-                    "io.sundr:builder-annotations:jar:sources:0.19.2",
-                    "io.sundr:sundr-codegen:jar:sources:0.19.2",
                     "org.apache.commons:commons-lang3:jar:sources:3.7",
                     "com.google.code.gson:gson:jar:sources:2.8.2",
                     "org.joda:joda-convert:jar:sources:1.2",
                     "com.squareup.okhttp3:logging-interceptor:jar:sources:3.14.3",
-                    "io.sundr:sundr-core:jar:sources:0.19.2",
                     "io.gsonfire:gson-fire:jar:sources:1.8.3",
+                    "io.sundr:builder-annotations:jar:sources:0.21.0",
                     "io.swagger:swagger-annotations:jar:sources:1.5.22",
                     "com.squareup.okhttp3:okhttp:jar:sources:3.14.3",
-                    "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
-                    "com.squareup.okio:okio:jar:sources:1.17.2"
+                    "javax.annotation:javax.annotation-api:jar:sources:1.3.2"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0-sources.jar",
-                    "https://maven.google.com/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0-sources.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-api/7.0.0/client-java-api-7.0.0-sources.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0-sources.jar",
+                    "https://maven.google.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0-sources.jar"
                 ],
-                "sha256": "09476d45d87c3a6e86e020994fc035b4c8f85b3858130f99a812e9baff7f242b"
+                "sha256": "d22b31f794ed5a75abdbc8ab7632f2bf49a68d4ef290aa88395f2f6749e3ef4f",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-api/8.0.0/client-java-api-8.0.0-sources.jar"
             },
             {
-                "coord": "io.kubernetes:client-java-proto:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0.jar",
-                "directDependencies": [
+                "coord": "io.kubernetes:client-java-proto:8.0.0",
+                "dependencies": [
                     "com.google.protobuf:protobuf-java:3.4.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.protobuf:protobuf-java:3.4.0"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0.jar",
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0.jar",
-                    "https://maven.google.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0.jar",
+                    "https://maven.google.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0.jar"
                 ],
-                "sha256": "ac8f632f87b522dbdfbf0b93bcb70e0c72965c9b603f90b3f96294c9b20c90a1"
+                "sha256": "730860866481f3dac5771bb774cdd0a1a4a8e2aa6d8b15d533875d71564623be",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0.jar"
             },
             {
-                "coord": "io.kubernetes:client-java-proto:jar:sources:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0-sources.jar",
-                "directDependencies": [
+                "coord": "io.kubernetes:client-java-proto:jar:sources:8.0.0",
+                "dependencies": [
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0-sources.jar",
-                    "https://maven.google.com/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0-sources.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-proto/7.0.0/client-java-proto-7.0.0-sources.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0-sources.jar",
+                    "https://maven.google.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0-sources.jar"
                 ],
-                "sha256": "639c7a4810d8ab62568fae2bddd7c417777be40fa7d4ef57206f5b0a4cb36e28"
+                "sha256": "c97699e7ce794d3ef9eb80310ce2d2003978d823877c0e35a03732649ad8f396",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java-proto/8.0.0/client-java-proto-8.0.0-sources.jar"
             },
             {
-                "coord": "io.kubernetes:client-java:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0.jar",
-                "directDependencies": [
-                    "org.yaml:snakeyaml:1.15",
-                    "org.slf4j:slf4j-api:1.7.30",
-                    "org.apache.commons:commons-compress:1.14",
-                    "commons-codec:commons-codec:1.11",
-                    "org.bouncycastle:bcpkix-jdk15on:1.61",
-                    "org.apache.commons:commons-lang3:3.7",
-                    "io.kubernetes:client-java-api:7.0.0",
-                    "org.apache.commons:commons-collections4:4.1",
-                    "com.google.protobuf:protobuf-java:3.4.0",
-                    "com.google.guava:guava:18.0",
-                    "org.bouncycastle:bcprov-ext-jdk15on:1.61",
-                    "io.kubernetes:client-java-proto:7.0.0"
-                ],
+                "coord": "io.kubernetes:client-java:8.0.0",
                 "dependencies": [
                     "io.swagger:swagger-annotations:1.5.22",
                     "joda-time:joda-time:2.9.3",
                     "org.yaml:snakeyaml:1.15",
                     "com.squareup.okhttp3:okhttp:3.14.3",
                     "com.google.code.findbugs:jsr305:3.0.2",
-                    "io.sundr:resourcecify-annotations:0.19.2",
+                    "io.sundr:sundr-core:0.21.0",
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.apache.commons:commons-compress:1.14",
-                    "io.sundr:sundr-core:0.19.2",
                     "commons-codec:commons-codec:1.11",
                     "org.bouncycastle:bcpkix-jdk15on:1.61",
                     "org.apache.commons:commons-lang3:3.7",
-                    "io.kubernetes:client-java-api:7.0.0",
+                    "io.sundr:builder-annotations:0.21.0",
                     "org.apache.commons:commons-collections4:4.1",
+                    "io.sundr:sundr-codegen:0.21.0",
                     "com.google.protobuf:protobuf-java:3.4.0",
                     "com.squareup.okhttp3:logging-interceptor:3.14.3",
                     "com.google.guava:guava:18.0",
-                    "io.sundr:sundr-codegen:0.19.2",
                     "org.joda:joda-convert:1.2",
+                    "io.sundr:resourcecify-annotations:0.21.0",
+                    "io.kubernetes:client-java-api:8.0.0",
                     "javax.annotation:javax.annotation-api:1.3.2",
-                    "io.sundr:builder-annotations:0.19.2",
                     "com.google.code.gson:gson:2.8.2",
                     "org.bouncycastle:bcprov-ext-jdk15on:1.61",
                     "com.squareup.okio:okio:1.17.2",
                     "org.bouncycastle:bcprov-jdk15on:jar:1.61",
                     "io.gsonfire:gson-fire:1.8.3",
-                    "io.kubernetes:client-java-proto:7.0.0"
+                    "io.kubernetes:client-java-proto:8.0.0",
+                    "org.bitbucket.b_c:jose4j:0.7.0"
+                ],
+                "directDependencies": [
+                    "org.yaml:snakeyaml:1.15",
+                    "org.slf4j:slf4j-api:1.7.30",
+                    "org.apache.commons:commons-compress:1.14",
+                    "commons-codec:commons-codec:1.11",
+                    "org.bouncycastle:bcpkix-jdk15on:1.61",
+                    "org.apache.commons:commons-lang3:3.7",
+                    "org.apache.commons:commons-collections4:4.1",
+                    "com.google.protobuf:protobuf-java:3.4.0",
+                    "com.google.guava:guava:18.0",
+                    "io.kubernetes:client-java-api:8.0.0",
+                    "org.bouncycastle:bcprov-ext-jdk15on:1.61",
+                    "io.kubernetes:client-java-proto:8.0.0",
+                    "org.bitbucket.b_c:jose4j:0.7.0"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0.jar",
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0.jar",
-                    "https://maven.google.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java/7.0.0/client-java-7.0.0.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0.jar",
+                    "https://maven.google.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java/8.0.0/client-java-8.0.0.jar"
                 ],
-                "sha256": "a25aa34c70b088cade3fea315c05cace896f7164c94ee41e94268197cb2fe7dc"
+                "sha256": "3680962144c792ce2cfbe49e3cafc432a1e17c55417e6b40afc1a0a0ecbf42e2",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0.jar"
             },
             {
-                "coord": "io.kubernetes:client-java:jar:sources:7.0.0",
-                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0-sources.jar",
-                "directDependencies": [
-                    "io.kubernetes:client-java-api:jar:sources:7.0.0",
-                    "org.slf4j:slf4j-api:jar:sources:1.7.30",
-                    "org.yaml:snakeyaml:jar:sources:1.15",
-                    "org.apache.commons:commons-lang3:jar:sources:3.7",
-                    "com.google.guava:guava:jar:sources:18.0",
-                    "io.kubernetes:client-java-proto:jar:sources:7.0.0",
-                    "org.apache.commons:commons-compress:jar:sources:1.14",
-                    "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
-                    "commons-codec:commons-codec:jar:sources:1.11",
-                    "org.apache.commons:commons-collections4:jar:sources:4.1",
-                    "org.bouncycastle:bcprov-ext-jdk15on:jar:sources:1.61",
-                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
-                ],
+                "coord": "io.kubernetes:client-java:jar:sources:8.0.0",
                 "dependencies": [
-                    "io.sundr:resourcecify-annotations:jar:sources:0.19.2",
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
                     "joda-time:joda-time:jar:sources:2.9.3",
-                    "io.sundr:builder-annotations:jar:sources:0.19.2",
-                    "io.kubernetes:client-java-api:jar:sources:7.0.0",
+                    "io.sundr:resourcecify-annotations:jar:sources:0.21.0",
+                    "io.sundr:sundr-codegen:jar:sources:0.21.0",
+                    "io.kubernetes:client-java-proto:jar:sources:8.0.0",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "org.yaml:snakeyaml:jar:sources:1.15",
-                    "io.sundr:sundr-codegen:jar:sources:0.19.2",
                     "org.apache.commons:commons-lang3:jar:sources:3.7",
                     "com.google.guava:guava:jar:sources:18.0",
-                    "io.kubernetes:client-java-proto:jar:sources:7.0.0",
                     "org.apache.commons:commons-compress:jar:sources:1.14",
                     "com.google.code.gson:gson:jar:sources:2.8.2",
                     "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
                     "org.joda:joda-convert:jar:sources:1.2",
                     "com.squareup.okhttp3:logging-interceptor:jar:sources:3.14.3",
-                    "io.sundr:sundr-core:jar:sources:0.19.2",
+                    "org.bitbucket.b_c:jose4j:jar:sources:0.7.0",
                     "io.gsonfire:gson-fire:jar:sources:1.8.3",
+                    "io.sundr:sundr-core:jar:sources:0.21.0",
                     "commons-codec:commons-codec:jar:sources:1.11",
+                    "io.kubernetes:client-java-api:jar:sources:8.0.0",
                     "org.apache.commons:commons-collections4:jar:sources:4.1",
                     "org.bouncycastle:bcprov-ext-jdk15on:jar:sources:1.61",
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
+                    "io.sundr:builder-annotations:jar:sources:0.21.0",
                     "org.bouncycastle:bcprov-jdk15on:jar:sources:1.61",
                     "io.swagger:swagger-annotations:jar:sources:1.5.22",
                     "com.squareup.okhttp3:okhttp:jar:sources:3.14.3",
                     "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
                     "com.squareup.okio:okio:jar:sources:1.17.2"
                 ],
-                "url": "https://jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0-sources.jar",
+                "directDependencies": [
+                    "io.kubernetes:client-java-proto:jar:sources:8.0.0",
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30",
+                    "org.yaml:snakeyaml:jar:sources:1.15",
+                    "org.apache.commons:commons-lang3:jar:sources:3.7",
+                    "com.google.guava:guava:jar:sources:18.0",
+                    "org.apache.commons:commons-compress:jar:sources:1.14",
+                    "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
+                    "org.bitbucket.b_c:jose4j:jar:sources:0.7.0",
+                    "commons-codec:commons-codec:jar:sources:1.11",
+                    "io.kubernetes:client-java-api:jar:sources:8.0.0",
+                    "org.apache.commons:commons-collections4:jar:sources:4.1",
+                    "org.bouncycastle:bcprov-ext-jdk15on:jar:sources:1.61",
+                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0-sources.jar",
-                    "https://maven.google.com/io/kubernetes/client-java/7.0.0/client-java-7.0.0-sources.jar",
-                    "https://repo1.maven.org/maven2/io/kubernetes/client-java/7.0.0/client-java-7.0.0-sources.jar"
+                    "https://jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0-sources.jar",
+                    "https://maven.google.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/kubernetes/client-java/8.0.0/client-java-8.0.0-sources.jar"
                 ],
-                "sha256": "4b901cf00ed9ef9bc0bde52c9246cf7e759d8b872fe4425833c5b0fc15bd6f67"
+                "sha256": "078bd7dcf01001b81ea7ddd52644459c8360a8e1c21dcbb3b43c7e869bf4f920",
+                "url": "https://jcenter.bintray.com/io/kubernetes/client-java/8.0.0/client-java-8.0.0-sources.jar"
             },
             {
                 "coord": "io.netty:netty-all:4.1.22.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar",
                     "https://maven.google.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar"
                 ],
-                "sha256": "cabf1beb5d0c6fd71ed8374a8f6e3c24c08ab04fd67c5e07f0732de3cdb7cd8a"
+                "sha256": "cabf1beb5d0c6fd71ed8374a8f6e3c24c08ab04fd67c5e07f0732de3cdb7cd8a",
+                "url": "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final.jar"
             },
             {
                 "coord": "io.netty:netty-all:jar:sources:4.1.22.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar"
                 ],
-                "sha256": "e16e02483de53882bc5309bc31308a41f45973d450ca18af6b59d90f9ab0ebe9"
+                "sha256": "e16e02483de53882bc5309bc31308a41f45973d450ca18af6b59d90f9ab0ebe9",
+                "url": "https://jcenter.bintray.com/io/netty/netty-all/4.1.22.Final/netty-all-4.1.22.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-buffer:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-common:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar"
                 ],
-                "sha256": "7efc8f98224c703ef09a409e5ddffbe14f5b4b6f527d3836c1647b4d9eff8cec"
+                "sha256": "7efc8f98224c703ef09a409e5ddffbe14f5b4b6f527d3836c1647b4d9eff8cec",
+                "url": "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-buffer:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "433db92fb6e0f8c14228ad7a5e12f69180e17604ad8553d7bce32f685ac0b273"
+                "sha256": "433db92fb6e0f8c14228ad7a5e12f69180e17604ad8553d7bce32f685ac0b273",
+                "url": "https://jcenter.bintray.com/io/netty/netty-buffer/4.1.48.Final/netty-buffer-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-codec:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:4.1.48.Final",
-                    "io.netty:netty-common:4.1.48.Final",
-                    "io.netty:netty-transport:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:4.1.48.Final",
                     "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:4.1.48.Final",
+                    "io.netty:netty-common:4.1.48.Final",
+                    "io.netty:netty-transport:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar"
                 ],
-                "sha256": "81b4c316163a591b4f74fd2dc23a3ea45359cb817d0a9c4fc7f37dc9edfdbea8"
+                "sha256": "81b4c316163a591b4f74fd2dc23a3ea45359cb817d0a9c4fc7f37dc9edfdbea8",
+                "url": "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-codec:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
-                    "io.netty:netty-common:jar:sources:4.1.48.Final",
-                    "io.netty:netty-transport:jar:sources:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
+                    "io.netty:netty-common:jar:sources:4.1.48.Final",
+                    "io.netty:netty-transport:jar:sources:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "c9ce166db31e2c5dce08b902c5ac7fe3427209c2d3fea63ef3532559923577c1"
+                "sha256": "c9ce166db31e2c5dce08b902c5ac7fe3427209c2d3fea63ef3532559923577c1",
+                "url": "https://jcenter.bintray.com/io/netty/netty-codec/4.1.48.Final/netty-codec-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-common:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar"
                 ],
-                "sha256": "e44a2369566fd1fa8a0f30b12e2801de8fb405b9d1fa3894a58b6262065a9916"
+                "sha256": "e44a2369566fd1fa8a0f30b12e2801de8fb405b9d1fa3894a58b6262065a9916",
+                "url": "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-common:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "6363f44bed7a8157633114665af102de531a15c7fb0ffc2f224ecd45c4ed49d1"
+                "sha256": "6363f44bed7a8157633114665af102de531a15c7fb0ffc2f224ecd45c4ed49d1",
+                "url": "https://jcenter.bintray.com/io/netty/netty-common/4.1.48.Final/netty-common-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-handler:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final",
                     "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-common:4.1.48.Final",
                     "io.netty:netty-codec:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final",
                     "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-common:4.1.48.Final",
                     "io.netty:netty-codec:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar"
                 ],
-                "sha256": "757f83c7891ad2ebad209f02d8dbca0121e03f7062c2d4ec9d00eba1a0d403d5"
+                "sha256": "757f83c7891ad2ebad209f02d8dbca0121e03f7062c2d4ec9d00eba1a0d403d5",
+                "url": "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-handler:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-codec:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport:jar:sources:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-codec:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "2ac4215797f4d661910d2a143223965e0ba956a7ca8a26259e4eab5cfe6a455a"
+                "sha256": "2ac4215797f4d661910d2a143223965e0ba956a7ca8a26259e4eab5cfe6a455a",
+                "url": "https://jcenter.bintray.com/io/netty/netty-handler/4.1.48.Final/netty-handler-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-resolver:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-common:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar"
                 ],
-                "sha256": "fb125914398ebef821def3dbb1642f9f360f39d182f00149ef3db845ebf06ad2"
+                "sha256": "fb125914398ebef821def3dbb1642f9f360f39d182f00149ef3db845ebf06ad2",
+                "url": "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-resolver:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "e0fd3b04e6d77b1020ed5ab321fe5747f4239310359d1f96f26e2d98a20cdc26"
+                "sha256": "e0fd3b04e6d77b1020ed5ab321fe5747f4239310359d1f96f26e2d98a20cdc26",
+                "url": "https://jcenter.bintray.com/io/netty/netty-resolver/4.1.48.Final/netty-resolver-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-tcnative-boringssl-static:2.0.7.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar",
                     "https://maven.google.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar"
                 ],
-                "sha256": "762aa9b16025e373f73534e8b78bf1552b60bfb1c2c9b579de49e9e5517680de"
+                "sha256": "762aa9b16025e373f73534e8b78bf1552b60bfb1c2c9b579de49e9e5517680de",
+                "url": "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final.jar"
             },
             {
                 "coord": "io.netty:netty-tcnative-boringssl-static:jar:sources:2.0.7.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar"
                 ],
-                "sha256": "839e9f4d630a568cdacdcc11b254c5a7b4c56f8ae32ec272c966a5692d475ab2"
+                "sha256": "839e9f4d630a568cdacdcc11b254c5a7b4c56f8ae32ec272c966a5692d475ab2",
+                "url": "https://jcenter.bintray.com/io/netty/netty-tcnative-boringssl-static/2.0.7.Final/netty-tcnative-boringssl-static-2.0.7.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-transport-native-epoll:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final",
+                    "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-common:4.1.48.Final",
-                    "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "dependencies": [
-                    "io.netty:netty-transport:4.1.48.Final",
+                "directDependencies": [
                     "io.netty:netty-buffer:4.1.48.Final",
-                    "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-common:4.1.48.Final",
+                    "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar",
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar"
                 ],
-                "sha256": "f16ed1dd7a2403cfcbba70e73348bcb8d572f12631f476e674e84b195be73744"
+                "sha256": "f16ed1dd7a2403cfcbba70e73348bcb8d572f12631f476e674e84b195be73744",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-transport-native-epoll:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
-                    "io.netty:netty-common:jar:sources:4.1.48.Final",
-                    "io.netty:netty-transport:jar:sources:4.1.48.Final",
-                    "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
@@ -2672,807 +2670,813 @@
                     "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
-                "mirror_urls": [
-                    "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
-                    "https://maven.google.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
-                    "https://repo1.maven.org/maven2/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar"
+                "directDependencies": [
+                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
+                    "io.netty:netty-common:jar:sources:4.1.48.Final",
+                    "io.netty:netty-transport:jar:sources:4.1.48.Final",
+                    "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
+                "mirror_urls": [
+                    "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
+                    "https://maven.google.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar",
+                    "https://repo1.maven.org/maven2/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "b675ba3800ff1ded76b6f90cfb3b98040a213c3c48f5073cd60092b15bbf6ec7"
+                "sha256": "b675ba3800ff1ded76b6f90cfb3b98040a213c3c48f5073cd60092b15bbf6ec7",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-epoll/4.1.48.Final/netty-transport-native-epoll-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-transport-native-unix-common:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:4.1.48.Final",
-                    "io.netty:netty-common:4.1.48.Final",
-                    "io.netty:netty-transport:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:4.1.48.Final",
                     "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-transport:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:4.1.48.Final",
+                    "io.netty:netty-common:4.1.48.Final",
+                    "io.netty:netty-transport:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar"
                 ],
-                "sha256": "c4142429437845d966babba5eddce47203e1f256209e455019d2538ebec58b95"
+                "sha256": "c4142429437845d966babba5eddce47203e1f256209e455019d2538ebec58b95",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
-                    "io.netty:netty-common:jar:sources:4.1.48.Final",
-                    "io.netty:netty-transport:jar:sources:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
+                    "io.netty:netty-common:jar:sources:4.1.48.Final",
+                    "io.netty:netty-transport:jar:sources:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "43bfcda404e6366fc73944f5105d6ad85104d52070b0c729fd1102a8f7136565"
+                "sha256": "43bfcda404e6366fc73944f5105d6ad85104d52070b0c729fd1102a8f7136565",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport-native-unix-common/4.1.48.Final/netty-transport-native-unix-common-4.1.48.Final-sources.jar"
             },
             {
                 "coord": "io.netty:netty-transport:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:4.1.48.Final",
-                    "io.netty:netty-common:4.1.48.Final",
-                    "io.netty:netty-resolver:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:4.1.48.Final",
                     "io.netty:netty-resolver:4.1.48.Final",
                     "io.netty:netty-buffer:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:4.1.48.Final",
+                    "io.netty:netty-common:4.1.48.Final",
+                    "io.netty:netty-resolver:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar",
                     "https://maven.google.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar"
                 ],
-                "sha256": "6b4ba9e09a8e060bad2540845491b5fa1ca73614d157860e657f4027c91e72fd"
+                "sha256": "6b4ba9e09a8e060bad2540845491b5fa1ca73614d157860e657f4027c91e72fd",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final.jar"
             },
             {
                 "coord": "io.netty:netty-transport:jar:sources:4.1.48.Final",
-                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar",
-                "directDependencies": [
-                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
-                    "io.netty:netty-common:jar:sources:4.1.48.Final",
-                    "io.netty:netty-resolver:jar:sources:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar",
+                "directDependencies": [
+                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
+                    "io.netty:netty-common:jar:sources:4.1.48.Final",
+                    "io.netty:netty-resolver:jar:sources:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar",
                     "https://maven.google.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar",
                     "https://repo1.maven.org/maven2/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar"
                 ],
-                "sha256": "9d973dfff21f3d41745e1189721fdd2e7d717b8e410d438041d27d2e08df94d1"
+                "sha256": "9d973dfff21f3d41745e1189721fdd2e7d717b8e410d438041d27d2e08df94d1",
+                "url": "https://jcenter.bintray.com/io/netty/netty-transport/4.1.48.Final/netty-transport-4.1.48.Final-sources.jar"
             },
             {
-                "coord": "io.sundr:builder-annotations:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2.jar",
-                "directDependencies": [
-                    "io.sundr:resourcecify-annotations:0.19.2",
-                    "io.sundr:sundr-codegen:0.19.2",
-                    "io.sundr:sundr-core:0.19.2"
-                ],
+                "coord": "io.sundr:builder-annotations:0.21.0",
                 "dependencies": [
-                    "io.sundr:sundr-core:0.19.2",
-                    "io.sundr:resourcecify-annotations:0.19.2",
-                    "io.sundr:sundr-codegen:0.19.2"
+                    "io.sundr:sundr-codegen:0.21.0",
+                    "io.sundr:sundr-core:0.21.0",
+                    "io.sundr:resourcecify-annotations:0.21.0"
+                ],
+                "directDependencies": [
+                    "io.sundr:resourcecify-annotations:0.21.0",
+                    "io.sundr:sundr-codegen:0.21.0",
+                    "io.sundr:sundr-core:0.21.0"
                 ],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2.jar",
-                    "https://maven.google.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2.jar"
+                    "https://jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0.jar",
+                    "https://maven.google.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0.jar"
                 ],
-                "sha256": "a0483c5d51b4c90d1245911a4a97ac398087aea7f4518d51fcadbfa02987f6ad"
+                "sha256": "ff8aff3afbdc4d5cc98cea05bb84bf864b98faed29b481e1f70d984482b4f605",
+                "url": "https://jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0.jar"
             },
             {
-                "coord": "io.sundr:builder-annotations:jar:sources:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2-sources.jar",
-                "directDependencies": [
-                    "io.sundr:resourcecify-annotations:jar:sources:0.19.2",
-                    "io.sundr:sundr-codegen:jar:sources:0.19.2",
-                    "io.sundr:sundr-core:jar:sources:0.19.2"
-                ],
+                "coord": "io.sundr:builder-annotations:jar:sources:0.21.0",
                 "dependencies": [
-                    "io.sundr:sundr-core:jar:sources:0.19.2",
-                    "io.sundr:resourcecify-annotations:jar:sources:0.19.2",
-                    "io.sundr:sundr-codegen:jar:sources:0.19.2"
+                    "io.sundr:sundr-core:jar:sources:0.21.0",
+                    "io.sundr:resourcecify-annotations:jar:sources:0.21.0",
+                    "io.sundr:sundr-codegen:jar:sources:0.21.0"
+                ],
+                "directDependencies": [
+                    "io.sundr:resourcecify-annotations:jar:sources:0.21.0",
+                    "io.sundr:sundr-codegen:jar:sources:0.21.0",
+                    "io.sundr:sundr-core:jar:sources:0.21.0"
                 ],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2-sources.jar",
-                    "https://maven.google.com/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2-sources.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/builder-annotations/0.19.2/builder-annotations-0.19.2-sources.jar"
+                    "https://jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0-sources.jar",
+                    "https://maven.google.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0-sources.jar"
                 ],
-                "sha256": "7fa9bd12f8a07cabdbc2139caf16da2ee0f1aa7cbe79887d32fc9639fc2afadf"
+                "sha256": "d6c50f681dd37d3909f0f2ce7171a336e75b3b23f119a359bf038498925db066",
+                "url": "https://jcenter.bintray.com/io/sundr/builder-annotations/0.21.0/builder-annotations-0.21.0-sources.jar"
             },
             {
-                "coord": "io.sundr:resourcecify-annotations:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:resourcecify-annotations:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2.jar",
-                    "https://maven.google.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2.jar"
+                    "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0.jar",
+                    "https://maven.google.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0.jar"
                 ],
-                "sha256": "d1875bd9d01e9af67258abdf9cad9596b238f8269b82191c6e2f4f0beec0430d"
+                "sha256": "bfb03d80456f76c1e32b11bd7ded59ab46bf26c0079bd37cff8093d4386d1081",
+                "url": "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0.jar"
             },
             {
-                "coord": "io.sundr:resourcecify-annotations:jar:sources:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2-sources.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:resourcecify-annotations:jar:sources:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2-sources.jar",
-                    "https://maven.google.com/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2-sources.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/resourcecify-annotations/0.19.2/resourcecify-annotations-0.19.2-sources.jar"
+                    "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0-sources.jar",
+                    "https://maven.google.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0-sources.jar"
                 ],
-                "sha256": "ca068a5a0d2d86d6f70a707c92f7ee218d85d69b8d98a2b523aa1c42d2f7ac21"
+                "sha256": "5f565f8299f4cc021b0b1efd93d7d411b6d45bab5e76637afa1099f563457541",
+                "url": "https://jcenter.bintray.com/io/sundr/resourcecify-annotations/0.21.0/resourcecify-annotations-0.21.0-sources.jar"
             },
             {
-                "coord": "io.sundr:sundr-codegen:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:sundr-codegen:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2.jar",
-                    "https://maven.google.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2.jar"
+                    "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0.jar",
+                    "https://maven.google.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0.jar"
                 ],
-                "sha256": "ede82207d5f3727ebfb4489bdb0e2d1cfc43bba355118494ab1e00d31ee50da3"
+                "sha256": "3dd7902f10b8c308d1d0217c5bfe1248365c60d8fdcda15639cad69209ecd8cb",
+                "url": "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0.jar"
             },
             {
-                "coord": "io.sundr:sundr-codegen:jar:sources:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2-sources.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:sundr-codegen:jar:sources:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2-sources.jar",
-                    "https://maven.google.com/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2-sources.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/sundr-codegen/0.19.2/sundr-codegen-0.19.2-sources.jar"
+                    "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0-sources.jar",
+                    "https://maven.google.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0-sources.jar"
                 ],
-                "sha256": "3fbe1591da58f354c4b715105d52f3fb58e324bf7a559b69a005e0ba8c551f00"
+                "sha256": "db4e56cf990124c490ced8157c615df1abb30ff21bc55b95ded8dc7e29f870b5",
+                "url": "https://jcenter.bintray.com/io/sundr/sundr-codegen/0.21.0/sundr-codegen-0.21.0-sources.jar"
             },
             {
-                "coord": "io.sundr:sundr-core:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:sundr-core:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2.jar",
-                    "https://maven.google.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2.jar"
+                    "https://jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0.jar",
+                    "https://maven.google.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0.jar"
                 ],
-                "sha256": "1d2677342e669f7d693a6dbe32a9ddad3217ee503058fd8414fcbb5523810eb8"
+                "sha256": "8c6a6dd85479d192f6acff660faee04d2857613d10afa97bdfba44f04178f2ad",
+                "url": "https://jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0.jar"
             },
             {
-                "coord": "io.sundr:sundr-core:jar:sources:0.19.2",
-                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2-sources.jar",
-                "directDependencies": [],
+                "coord": "io.sundr:sundr-core:jar:sources:0.21.0",
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "com.sun:tools"
                 ],
-                "url": "https://jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0-sources.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2-sources.jar",
-                    "https://maven.google.com/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2-sources.jar",
-                    "https://repo1.maven.org/maven2/io/sundr/sundr-core/0.19.2/sundr-core-0.19.2-sources.jar"
+                    "https://jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0-sources.jar",
+                    "https://maven.google.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0-sources.jar",
+                    "https://repo1.maven.org/maven2/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0-sources.jar"
                 ],
-                "sha256": "26ecfe115cce2513affcf64e7baf65b16f65758415853ee849ab047fff78c4ac"
+                "sha256": "1ef356fb62aee8b203d10d48e199ffe532560e976f867e6088a463380c3a538b",
+                "url": "https://jcenter.bintray.com/io/sundr/sundr-core/0.21.0/sundr-core-0.21.0-sources.jar"
             },
             {
                 "coord": "io.swagger:swagger-annotations:1.5.22",
-                "file": "v1/https/jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar",
                     "https://maven.google.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar",
                     "https://repo1.maven.org/maven2/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar"
                 ],
-                "sha256": "e3f7af74982bce1c4ac1c73a4d6d1efc2fabc9811b2f50679069a65a873e753c"
+                "sha256": "e3f7af74982bce1c4ac1c73a4d6d1efc2fabc9811b2f50679069a65a873e753c",
+                "url": "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22.jar"
             },
             {
                 "coord": "io.swagger:swagger-annotations:jar:sources:1.5.22",
-                "file": "v1/https/jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar",
                     "https://maven.google.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar",
                     "https://repo1.maven.org/maven2/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar"
                 ],
-                "sha256": "687b3feeb465c143e225d15d25363f553252e8e388679b47c275de072cb1fbce"
+                "sha256": "687b3feeb465c143e225d15d25363f553252e8e388679b47c275de072cb1fbce",
+                "url": "https://jcenter.bintray.com/io/swagger/swagger-annotations/1.5.22/swagger-annotations-1.5.22-sources.jar"
             },
             {
                 "coord": "it.unimi.dsi:fastutil:8.1.1",
-                "file": "v1/https/jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar",
                     "https://maven.google.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar",
                     "https://repo1.maven.org/maven2/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar"
                 ],
-                "sha256": "d9cdf8d69e1abde5d6ee509977340c29669f672ed9cc03ccff4d4f1981c071b1"
+                "sha256": "d9cdf8d69e1abde5d6ee509977340c29669f672ed9cc03ccff4d4f1981c071b1",
+                "url": "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1.jar"
             },
             {
                 "coord": "it.unimi.dsi:fastutil:jar:sources:8.1.1",
-                "file": "v1/https/jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar",
                     "https://maven.google.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar",
                     "https://repo1.maven.org/maven2/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar"
                 ],
-                "sha256": "0262568658d3972d152b956b44ae351e9203bc323527052afd6a7ff5122f5d6c"
+                "sha256": "0262568658d3972d152b956b44ae351e9203bc323527052afd6a7ff5122f5d6c",
+                "url": "https://jcenter.bintray.com/it/unimi/dsi/fastutil/8.1.1/fastutil-8.1.1-sources.jar"
             },
             {
                 "coord": "javax.activation:activation:1.1.1",
-                "file": "v1/https/jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1.jar",
                     "https://maven.google.com/javax/activation/activation/1.1.1/activation-1.1.1.jar",
                     "https://repo1.maven.org/maven2/javax/activation/activation/1.1.1/activation-1.1.1.jar"
                 ],
-                "sha256": "ae475120e9fcd99b4b00b38329bd61cdc5eb754eee03fe66c01f50e137724f99"
+                "sha256": "ae475120e9fcd99b4b00b38329bd61cdc5eb754eee03fe66c01f50e137724f99",
+                "url": "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1.jar"
             },
             {
                 "coord": "javax.activation:activation:jar:sources:1.1.1",
-                "file": "v1/https/jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar",
                     "https://maven.google.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar",
                     "https://repo1.maven.org/maven2/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar"
                 ],
-                "sha256": "8f0625a411700ec64163f8d4bba860475519acb9799f47139c7f49740fd93703"
+                "sha256": "8f0625a411700ec64163f8d4bba860475519acb9799f47139c7f49740fd93703",
+                "url": "https://jcenter.bintray.com/javax/activation/activation/1.1.1/activation-1.1.1-sources.jar"
             },
             {
                 "coord": "javax.annotation:javax.annotation-api:1.3.2",
-                "file": "v1/https/jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar",
                     "https://maven.google.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar",
                     "https://repo1.maven.org/maven2/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar"
                 ],
-                "sha256": "e04ba5195bcd555dc95650f7cc614d151e4bcd52d29a10b8aa2197f3ab89ab9b"
+                "sha256": "e04ba5195bcd555dc95650f7cc614d151e4bcd52d29a10b8aa2197f3ab89ab9b",
+                "url": "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar"
             },
             {
                 "coord": "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
-                "file": "v1/https/jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar",
                     "https://maven.google.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar",
                     "https://repo1.maven.org/maven2/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar"
                 ],
-                "sha256": "128971e52e0d84a66e3b6e049dab8ad7b2c58b7e1ad37fa2debd3d40c2947b95"
+                "sha256": "128971e52e0d84a66e3b6e049dab8ad7b2c58b7e1ad37fa2debd3d40c2947b95",
+                "url": "https://jcenter.bintray.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2-sources.jar"
             },
             {
                 "coord": "javax.inject:javax.inject:1",
-                "file": "v1/https/jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1.jar",
                     "https://maven.google.com/javax/inject/javax.inject/1/javax.inject-1.jar",
                     "https://repo1.maven.org/maven2/javax/inject/javax.inject/1/javax.inject-1.jar"
                 ],
-                "sha256": "91c77044a50c481636c32d916fd89c9118a72195390452c81065080f957de7ff"
+                "sha256": "91c77044a50c481636c32d916fd89c9118a72195390452c81065080f957de7ff",
+                "url": "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1.jar"
             },
             {
                 "coord": "javax.inject:javax.inject:jar:sources:1",
-                "file": "v1/https/jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar",
                     "https://maven.google.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar",
                     "https://repo1.maven.org/maven2/javax/inject/javax.inject/1/javax.inject-1-sources.jar"
                 ],
-                "sha256": "c4b87ee2911c139c3daf498a781967f1eb2e75bc1a8529a2e7b328a15d0e433e"
+                "sha256": "c4b87ee2911c139c3daf498a781967f1eb2e75bc1a8529a2e7b328a15d0e433e",
+                "url": "https://jcenter.bintray.com/javax/inject/javax.inject/1/javax.inject-1-sources.jar"
             },
             {
                 "coord": "javax.servlet:javax.servlet-api:3.1.0",
-                "file": "v1/https/jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar",
                     "https://maven.google.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar",
                     "https://repo1.maven.org/maven2/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar"
                 ],
-                "sha256": "af456b2dd41c4e82cf54f3e743bc678973d9fe35bd4d3071fa05c7e5333b8482"
+                "sha256": "af456b2dd41c4e82cf54f3e743bc678973d9fe35bd4d3071fa05c7e5333b8482",
+                "url": "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar"
             },
             {
                 "coord": "javax.servlet:javax.servlet-api:jar:sources:3.1.0",
-                "file": "v1/https/jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar",
                     "https://maven.google.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar",
                     "https://repo1.maven.org/maven2/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar"
                 ],
-                "sha256": "5c6d640f01e8e7ffdba21b2b75c0f64f0c30fd1fc3372123750c034cb363012a"
+                "sha256": "5c6d640f01e8e7ffdba21b2b75c0f64f0c30fd1fc3372123750c034cb363012a",
+                "url": "https://jcenter.bintray.com/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0-sources.jar"
             },
             {
                 "coord": "javax.validation:validation-api:1.1.0.Final",
-                "file": "v1/https/jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar",
                     "https://maven.google.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar",
                     "https://repo1.maven.org/maven2/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar"
                 ],
-                "sha256": "f39d7ba7253e35f5ac48081ec1bc28c5df9b32ac4b7db20853e5a8e76bf7b0ed"
+                "sha256": "f39d7ba7253e35f5ac48081ec1bc28c5df9b32ac4b7db20853e5a8e76bf7b0ed",
+                "url": "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final.jar"
             },
             {
                 "coord": "javax.validation:validation-api:jar:sources:1.1.0.Final",
-                "file": "v1/https/jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar",
                     "https://maven.google.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar",
                     "https://repo1.maven.org/maven2/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar"
                 ],
-                "sha256": "30c421ac184ae6204961304b60cd06db5fde3db5159e81430bcdcb386cf19470"
+                "sha256": "30c421ac184ae6204961304b60cd06db5fde3db5159e81430bcdcb386cf19470",
+                "url": "https://jcenter.bintray.com/javax/validation/validation-api/1.1.0.Final/validation-api-1.1.0.Final-sources.jar"
             },
             {
                 "coord": "javax.ws.rs:javax.ws.rs-api:2.0.1",
-                "file": "v1/https/jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar",
                     "https://maven.google.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar",
                     "https://repo1.maven.org/maven2/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar"
                 ],
-                "sha256": "38607d626f2288d8fbc1b1f8a62c369e63806d9a313ac7cbc5f9d6c94f4b466d"
+                "sha256": "38607d626f2288d8fbc1b1f8a62c369e63806d9a313ac7cbc5f9d6c94f4b466d",
+                "url": "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1.jar"
             },
             {
                 "coord": "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                "file": "v1/https/jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar",
                     "https://maven.google.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar",
                     "https://repo1.maven.org/maven2/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar"
                 ],
-                "sha256": "e5195bd059fe5b6f46f376bf52e2bc230b017a3975ec586dee4fbaca7ec02206"
+                "sha256": "e5195bd059fe5b6f46f376bf52e2bc230b017a3975ec586dee4fbaca7ec02206",
+                "url": "https://jcenter.bintray.com/javax/ws/rs/javax.ws.rs-api/2.0.1/javax.ws.rs-api-2.0.1-sources.jar"
             },
             {
                 "coord": "javax.xml.bind:jaxb-api:2.3.0",
-                "file": "v1/https/jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar",
                     "https://maven.google.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar",
                     "https://repo1.maven.org/maven2/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar"
                 ],
-                "sha256": "883007989d373d19f352ba9792b25dec21dc7d0e205a710a93a3815101bb3d03"
+                "sha256": "883007989d373d19f352ba9792b25dec21dc7d0e205a710a93a3815101bb3d03",
+                "url": "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0.jar"
             },
             {
                 "coord": "javax.xml.bind:jaxb-api:jar:sources:2.3.0",
-                "file": "v1/https/jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar",
                     "https://maven.google.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar",
                     "https://repo1.maven.org/maven2/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar"
                 ],
-                "sha256": "de09da7ffa14a4d6d35bde4452b3a88fafa1b8aa5aca0d5bad427649069e8b9f"
+                "sha256": "de09da7ffa14a4d6d35bde4452b3a88fafa1b8aa5aca0d5bad427649069e8b9f",
+                "url": "https://jcenter.bintray.com/javax/xml/bind/jaxb-api/2.3.0/jaxb-api-2.3.0-sources.jar"
             },
             {
                 "coord": "jline:jline:0.9.94",
-                "file": "v1/https/jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94.jar",
-                "directDependencies": [
-                    "junit:junit:4.12"
-                ],
                 "dependencies": [
                     "org.hamcrest:hamcrest-core:1.3",
                     "junit:junit:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94.jar",
+                "directDependencies": [
+                    "junit:junit:4.12"
+                ],
+                "file": "v1/https/jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94.jar",
                     "https://maven.google.com/jline/jline/0.9.94/jline-0.9.94.jar",
                     "https://repo1.maven.org/maven2/jline/jline/0.9.94/jline-0.9.94.jar"
                 ],
-                "sha256": "d8df0ffb12d87ca876271cda4d59b3feb94123882c1be1763b7faf2e0a0b0cbb"
+                "sha256": "d8df0ffb12d87ca876271cda4d59b3feb94123882c1be1763b7faf2e0a0b0cbb",
+                "url": "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94.jar"
             },
             {
                 "coord": "jline:jline:jar:sources:0.9.94",
-                "file": "v1/https/jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94-sources.jar",
-                "directDependencies": [
-                    "junit:junit:jar:sources:4.12"
-                ],
                 "dependencies": [
                     "org.hamcrest:hamcrest-core:jar:sources:1.3",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94-sources.jar",
+                "directDependencies": [
+                    "junit:junit:jar:sources:4.12"
+                ],
+                "file": "v1/https/jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94-sources.jar",
                     "https://maven.google.com/jline/jline/0.9.94/jline-0.9.94-sources.jar",
                     "https://repo1.maven.org/maven2/jline/jline/0.9.94/jline-0.9.94-sources.jar"
                 ],
-                "sha256": "e2efd2f925e717bb7e88997feb48c7ba2dfd02261051474b728eae58d38ae78b"
+                "sha256": "e2efd2f925e717bb7e88997feb48c7ba2dfd02261051474b728eae58d38ae78b",
+                "url": "https://jcenter.bintray.com/jline/jline/0.9.94/jline-0.9.94-sources.jar"
             },
             {
                 "coord": "joda-time:joda-time:2.9.3",
-                "file": "v1/https/jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar",
                     "https://maven.google.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar",
                     "https://repo1.maven.org/maven2/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar"
                 ],
-                "sha256": "a05f5b8b021802a71919b18702aebdf286148188b3ee9d26e6ec40e8d0071487"
+                "sha256": "a05f5b8b021802a71919b18702aebdf286148188b3ee9d26e6ec40e8d0071487",
+                "url": "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3.jar"
             },
             {
                 "coord": "joda-time:joda-time:jar:sources:2.9.3",
-                "file": "v1/https/jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar",
                     "https://maven.google.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar",
                     "https://repo1.maven.org/maven2/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar"
                 ],
-                "sha256": "c3d4baebf551cff03c518cbb4a60e78bcbde70287aba73e34a5d37ba3f5476a4"
+                "sha256": "c3d4baebf551cff03c518cbb4a60e78bcbde70287aba73e34a5d37ba3f5476a4",
+                "url": "https://jcenter.bintray.com/joda-time/joda-time/2.9.3/joda-time-2.9.3-sources.jar"
             },
             {
                 "coord": "junit:junit:4.12",
-                "file": "v1/https/jcenter.bintray.com/junit/junit/4.12/junit-4.12.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.hamcrest:hamcrest-core:1.3"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.hamcrest:hamcrest-core:1.3"
                 ],
-                "url": "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12.jar",
+                "file": "v1/https/jcenter.bintray.com/junit/junit/4.12/junit-4.12.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12.jar",
                     "https://maven.google.com/junit/junit/4.12/junit-4.12.jar",
                     "https://repo1.maven.org/maven2/junit/junit/4.12/junit-4.12.jar"
                 ],
-                "sha256": "59721f0805e223d84b90677887d9ff567dc534d7c502ca903c0c2b17f05c116a"
+                "sha256": "59721f0805e223d84b90677887d9ff567dc534d7c502ca903c0c2b17f05c116a",
+                "url": "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12.jar"
             },
             {
                 "coord": "junit:junit:jar:sources:4.12",
-                "file": "v1/https/jcenter.bintray.com/junit/junit/4.12/junit-4.12-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.hamcrest:hamcrest-core:jar:sources:1.3"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.hamcrest:hamcrest-core:jar:sources:1.3"
                 ],
-                "url": "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/junit/junit/4.12/junit-4.12-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12-sources.jar",
                     "https://maven.google.com/junit/junit/4.12/junit-4.12-sources.jar",
                     "https://repo1.maven.org/maven2/junit/junit/4.12/junit-4.12-sources.jar"
                 ],
-                "sha256": "9f43fea92033ad82bcad2ae44cec5c82abc9d6ee4b095cab921d11ead98bf2ff"
+                "sha256": "9f43fea92033ad82bcad2ae44cec5c82abc9d6ee4b095cab921d11ead98bf2ff",
+                "url": "https://jcenter.bintray.com/junit/junit/4.12/junit-4.12-sources.jar"
             },
             {
                 "coord": "log4j:log4j:1.2.17",
-                "file": "v1/https/jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17.jar",
                     "https://maven.google.com/log4j/log4j/1.2.17/log4j-1.2.17.jar",
                     "https://repo1.maven.org/maven2/log4j/log4j/1.2.17/log4j-1.2.17.jar"
                 ],
-                "sha256": "1d31696445697720527091754369082a6651bd49781b6005deb94e56753406f9"
+                "sha256": "1d31696445697720527091754369082a6651bd49781b6005deb94e56753406f9",
+                "url": "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17.jar"
             },
             {
                 "coord": "log4j:log4j:jar:sources:1.2.17",
-                "file": "v1/https/jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar",
                     "https://maven.google.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar",
                     "https://repo1.maven.org/maven2/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar"
                 ],
-                "sha256": "4d9ba787af1692aa88417c2a47a37a98125d645b91ab556252dbee0f45225493"
+                "sha256": "4d9ba787af1692aa88417c2a47a37a98125d645b91ab556252dbee0f45225493",
+                "url": "https://jcenter.bintray.com/log4j/log4j/1.2.17/log4j-1.2.17-sources.jar"
             },
             {
                 "coord": "net.java.dev.jets3t:jets3t:0.7.1",
-                "file": "v1/https/jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:1.11",
-                    "commons-httpclient:commons-httpclient:3.1",
-                    "commons-logging:commons-logging:1.2"
-                ],
                 "dependencies": [
                     "commons-httpclient:commons-httpclient:3.1",
                     "commons-logging:commons-logging:1.2",
                     "commons-codec:commons-codec:1.11"
                 ],
-                "url": "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar",
+                "directDependencies": [
+                    "commons-codec:commons-codec:1.11",
+                    "commons-httpclient:commons-httpclient:3.1",
+                    "commons-logging:commons-logging:1.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar",
                     "https://maven.google.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar",
                     "https://repo1.maven.org/maven2/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar"
                 ],
-                "sha256": "cf3c047571471682a7f59df54808a7143d78c6b8f6826809ee95a2f9637b7e60"
+                "sha256": "cf3c047571471682a7f59df54808a7143d78c6b8f6826809ee95a2f9637b7e60",
+                "url": "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar"
             },
             {
                 "coord": "net.java.dev.jets3t:jets3t:jar:sources:0.7.1",
-                "file": "v1/https/jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:jar:sources:1.11",
+                "dependencies": [
                     "commons-httpclient:commons-httpclient:jar:sources:3.1",
+                    "commons-codec:commons-codec:jar:sources:1.11",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "dependencies": [
-                    "commons-httpclient:commons-httpclient:jar:sources:3.1",
+                "directDependencies": [
                     "commons-codec:commons-codec:jar:sources:1.11",
+                    "commons-httpclient:commons-httpclient:jar:sources:3.1",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar",
                     "https://maven.google.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar",
                     "https://repo1.maven.org/maven2/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar"
                 ],
-                "sha256": "a530a8994182272306528c1ba3a3ac0688102585737f192398abbf7126ba4665"
+                "sha256": "a530a8994182272306528c1ba3a3ac0688102585737f192398abbf7126ba4665",
+                "url": "https://jcenter.bintray.com/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1-sources.jar"
             },
             {
                 "coord": "net.jcip:jcip-annotations:1.0",
-                "file": "v1/https/jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar",
                     "https://maven.google.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar",
                     "https://repo1.maven.org/maven2/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar"
                 ],
-                "sha256": "be5805392060c71474bf6c9a67a099471274d30b83eef84bfc4e0889a4f1dcc0"
+                "sha256": "be5805392060c71474bf6c9a67a099471274d30b83eef84bfc4e0889a4f1dcc0",
+                "url": "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar"
             },
             {
                 "coord": "net.jcip:jcip-annotations:jar:sources:1.0",
-                "file": "v1/https/jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar",
                     "https://maven.google.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar",
                     "https://repo1.maven.org/maven2/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar"
                 ],
-                "sha256": "e3ad6ae439e3cf8a25372de838efaa1a95f8ef9b5053d5d94fafe89c8c09814e"
+                "sha256": "e3ad6ae439e3cf8a25372de838efaa1a95f8ef9b5053d5d94fafe89c8c09814e",
+                "url": "https://jcenter.bintray.com/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0-sources.jar"
             },
             {
                 "coord": "net.jpountz.lz4:lz4:1.3.0",
-                "file": "v1/https/jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar",
                     "https://maven.google.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar",
                     "https://repo1.maven.org/maven2/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar"
                 ],
-                "sha256": "b877a4d4a3a0140486d3d0f83d9058e7c0ff6ca80b00d2f7b77145935b385b56"
+                "sha256": "b877a4d4a3a0140486d3d0f83d9058e7c0ff6ca80b00d2f7b77145935b385b56",
+                "url": "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0.jar"
             },
             {
                 "coord": "net.jpountz.lz4:lz4:jar:sources:1.3.0",
-                "file": "v1/https/jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar",
                     "https://maven.google.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar",
                     "https://repo1.maven.org/maven2/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar"
                 ],
-                "sha256": "956df26ca2bca3131a578688e300daeaef4e99e794516bdbcc5272e804c6f4ee"
+                "sha256": "956df26ca2bca3131a578688e300daeaef4e99e794516bdbcc5272e804c6f4ee",
+                "url": "https://jcenter.bintray.com/net/jpountz/lz4/lz4/1.3.0/lz4-1.3.0-sources.jar"
             },
             {
                 "coord": "net.minidev:accessors-smart:1.1",
-                "file": "v1/https/jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar",
+                "file": "v1/https/jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar",
                     "https://maven.google.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar",
                     "https://repo1.maven.org/maven2/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar"
                 ],
-                "sha256": "e6e04753913546da3ff0fbf532ac2831d0266f69246b1e6e295ba367aa9f02a5"
+                "sha256": "e6e04753913546da3ff0fbf532ac2831d0266f69246b1e6e295ba367aa9f02a5",
+                "url": "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1.jar"
             },
             {
                 "coord": "net.minidev:accessors-smart:jar:sources:1.1",
-                "file": "v1/https/jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar",
                     "https://maven.google.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar",
                     "https://repo1.maven.org/maven2/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar"
                 ],
-                "sha256": "9fba3d176840ae2744ff7cea1d87f3be38cc53b6dcae0781673b5317890b3dd4"
+                "sha256": "9fba3d176840ae2744ff7cea1d87f3be38cc53b6dcae0781673b5317890b3dd4",
+                "url": "https://jcenter.bintray.com/net/minidev/accessors-smart/1.1/accessors-smart-1.1-sources.jar"
             },
             {
                 "coord": "net.minidev:json-smart:2.2",
-                "file": "v1/https/jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2.jar",
-                "directDependencies": [
-                    "net.minidev:accessors-smart:1.1"
-                ],
                 "dependencies": [
                     "net.minidev:accessors-smart:1.1",
                     "org.ow2.asm:asm:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2.jar",
+                "directDependencies": [
+                    "net.minidev:accessors-smart:1.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2.jar",
                     "https://maven.google.com/net/minidev/json-smart/2.2/json-smart-2.2.jar",
                     "https://repo1.maven.org/maven2/net/minidev/json-smart/2.2/json-smart-2.2.jar"
                 ],
-                "sha256": "758d1b10c455a79991d10e2cf178ed89335290a5dcacd5483678ea626c0bd6e4"
+                "sha256": "758d1b10c455a79991d10e2cf178ed89335290a5dcacd5483678ea626c0bd6e4",
+                "url": "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2.jar"
             },
             {
                 "coord": "net.minidev:json-smart:jar:sources:2.2",
-                "file": "v1/https/jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar",
-                "directDependencies": [
-                    "net.minidev:accessors-smart:jar:sources:1.1"
-                ],
                 "dependencies": [
                     "net.minidev:accessors-smart:jar:sources:1.1",
                     "org.ow2.asm:asm:jar:sources:5.0.4"
                 ],
-                "url": "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar",
+                "directDependencies": [
+                    "net.minidev:accessors-smart:jar:sources:1.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar",
                     "https://maven.google.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar",
                     "https://repo1.maven.org/maven2/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar"
                 ],
-                "sha256": "25d4298eeeea14b3cb0160afd63212090efe98164db5c7a7dbc33e8a9a22ada8"
+                "sha256": "25d4298eeeea14b3cb0160afd63212090efe98164db5c7a7dbc33e8a9a22ada8",
+                "url": "https://jcenter.bintray.com/net/minidev/json-smart/2.2/json-smart-2.2-sources.jar"
             },
             {
                 "coord": "net.sf.kosmosfs:kfs:0.3",
-                "file": "v1/https/jcenter.bintray.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar",
                     "https://maven.google.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar",
                     "https://repo1.maven.org/maven2/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar"
                 ],
-                "sha256": "1807cbffb6c8876cf440b8f7d748fff49cea40ddd2d042bbb62bae3c87ba8842"
+                "sha256": "1807cbffb6c8876cf440b8f7d748fff49cea40ddd2d042bbb62bae3c87ba8842",
+                "url": "https://jcenter.bintray.com/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar"
             },
             {
                 "coord": "org.antlr:antlr4-runtime:4.5.2-1",
-                "file": "v1/https/jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar",
                     "https://maven.google.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar",
                     "https://repo1.maven.org/maven2/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar"
                 ],
-                "sha256": "e831413004bceed7d915c3a175927b1daabc4974b7b8a6f87bbce886d3550398"
+                "sha256": "e831413004bceed7d915c3a175927b1daabc4974b7b8a6f87bbce886d3550398",
+                "url": "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1.jar"
             },
             {
                 "coord": "org.antlr:antlr4-runtime:jar:sources:4.5.2-1",
-                "file": "v1/https/jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar",
                     "https://maven.google.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar",
                     "https://repo1.maven.org/maven2/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar"
                 ],
-                "sha256": "490bee7f6044bf0e014934ead9d292cae01e83800b0e08946f5a21ddacba5ed1"
+                "sha256": "490bee7f6044bf0e014934ead9d292cae01e83800b0e08946f5a21ddacba5ed1",
+                "url": "https://jcenter.bintray.com/org/antlr/antlr4-runtime/4.5.2-1/antlr4-runtime-4.5.2-1-sources.jar"
             },
             {
                 "coord": "org.apache.avro:avro:1.7.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
                     "com.thoughtworks.paranamer:paranamer:2.3",
                     "org.apache.commons:commons-compress:1.14",
@@ -3480,7 +3484,7 @@
                     "org.codehaus.jackson:jackson-mapper-asl:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:1.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
                     "com.thoughtworks.paranamer:paranamer:2.3",
                     "org.apache.commons:commons-compress:1.14",
@@ -3488,18 +3492,18 @@
                     "org.codehaus.jackson:jackson-mapper-asl:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:1.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar",
                     "https://maven.google.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar",
                     "https://repo1.maven.org/maven2/org/apache/avro/avro/1.7.4/avro-1.7.4.jar"
                 ],
-                "sha256": "a01d26e9a5ed0754e8c88dbb373fba896c57df0a0c424185767a3857855bb222"
+                "sha256": "a01d26e9a5ed0754e8c88dbb373fba896c57df0a0c424185767a3857855bb222",
+                "url": "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4.jar"
             },
             {
                 "coord": "org.apache.avro:avro:jar:sources:1.7.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8",
                     "com.thoughtworks.paranamer:paranamer:jar:sources:2.3",
@@ -3507,7 +3511,7 @@
                     "org.xerial.snappy:snappy-java:jar:sources:1.1.7.2",
                     "org.apache.commons:commons-compress:jar:sources:1.14"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8",
                     "com.thoughtworks.paranamer:paranamer:jar:sources:2.3",
@@ -3515,160 +3519,155 @@
                     "org.xerial.snappy:snappy-java:jar:sources:1.1.7.2",
                     "org.apache.commons:commons-compress:jar:sources:1.14"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar",
                     "https://maven.google.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar"
                 ],
-                "sha256": "cecd0753fec86ad7c0a6277eaeaae65d403383b93f0d35c4dbcbfe9886079915"
+                "sha256": "cecd0753fec86ad7c0a6277eaeaae65d403383b93f0d35c4dbcbfe9886079915",
+                "url": "https://jcenter.bintray.com/org/apache/avro/avro/1.7.4/avro-1.7.4-sources.jar"
             },
             {
                 "coord": "org.apache.commons:commons-collections4:4.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar",
                     "https://maven.google.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar"
                 ],
-                "sha256": "b1fe8b5968b57d8465425357ed2d9dc695504518bed2df5b565c4b8e68c1c8a5"
+                "sha256": "b1fe8b5968b57d8465425357ed2d9dc695504518bed2df5b565c4b8e68c1c8a5",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1.jar"
             },
             {
                 "coord": "org.apache.commons:commons-collections4:jar:sources:4.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar",
                     "https://maven.google.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar"
                 ],
-                "sha256": "21ab0a45f827629905b0ffc0f21cc9ae9ab727138dec8f20ec9b2f05869734c3"
+                "sha256": "21ab0a45f827629905b0ffc0f21cc9ae9ab727138dec8f20ec9b2f05869734c3",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-collections4/4.1/commons-collections4-4.1-sources.jar"
             },
             {
                 "coord": "org.apache.commons:commons-compress:1.14",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar",
                     "https://maven.google.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar"
                 ],
-                "sha256": "ad2c50c007e3384b7b46849d7f86a9ede873132256a78179f18b70e3adde7c99"
+                "sha256": "ad2c50c007e3384b7b46849d7f86a9ede873132256a78179f18b70e3adde7c99",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14.jar"
             },
             {
                 "coord": "org.apache.commons:commons-compress:jar:sources:1.14",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar",
                     "https://maven.google.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar"
                 ],
-                "sha256": "58c1eb4b545090cdd0de2f4c5ee7b0cbf057439ed607e2206d8a3530a1973010"
+                "sha256": "58c1eb4b545090cdd0de2f4c5ee7b0cbf057439ed607e2206d8a3530a1973010",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-compress/1.14/commons-compress-1.14-sources.jar"
             },
             {
                 "coord": "org.apache.commons:commons-lang3:3.7",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar",
                     "https://maven.google.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar"
                 ],
-                "sha256": "6e8dc31e046508d9953c96534edf0c2e0bfe6f468966b5b842b3f87e43b6a847"
+                "sha256": "6e8dc31e046508d9953c96534edf0c2e0bfe6f468966b5b842b3f87e43b6a847",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7.jar"
             },
             {
                 "coord": "org.apache.commons:commons-lang3:jar:sources:3.7",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar",
                     "https://maven.google.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar"
                 ],
-                "sha256": "8c48529d61778cee7b44e1c5858c887abed5395f5d13d6d52931d14ab22f017f"
+                "sha256": "8c48529d61778cee7b44e1c5858c887abed5395f5d13d6d52931d14ab22f017f",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-lang3/3.7/commons-lang3-3.7-sources.jar"
             },
             {
                 "coord": "org.apache.commons:commons-math3:3.6.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar",
                     "https://maven.google.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar"
                 ],
-                "sha256": "1e56d7b058d28b65abd256b8458e3885b674c1d588fa43cd7d1cbb9c7ef2b308"
+                "sha256": "1e56d7b058d28b65abd256b8458e3885b674c1d588fa43cd7d1cbb9c7ef2b308",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar"
             },
             {
                 "coord": "org.apache.commons:commons-math3:jar:sources:3.6.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar",
                     "https://maven.google.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar"
                 ],
-                "sha256": "e2ff85a3c360d56c51a7021614a194f3fbaf224054642ac535016f118322934d"
+                "sha256": "e2ff85a3c360d56c51a7021614a194f3fbaf224054642ac535016f118322934d",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1-sources.jar"
             },
             {
                 "coord": "org.apache.commons:commons-text:1.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.commons:commons-lang3:3.7"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.commons:commons-lang3:3.7"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar",
                     "https://maven.google.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-text/1.1/commons-text-1.1.jar"
                 ],
-                "sha256": "0149a81b421e646646d8724e0500f71264893c790aa2a771c056a1e984f02434"
+                "sha256": "0149a81b421e646646d8724e0500f71264893c790aa2a771c056a1e984f02434",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1.jar"
             },
             {
                 "coord": "org.apache.commons:commons-text:jar:sources:1.1",
-                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.commons:commons-lang3:jar:sources:3.7"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.commons:commons-lang3:jar:sources:3.7"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar",
                     "https://maven.google.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar"
                 ],
-                "sha256": "3f24065f7d4b847951e9e098cbb2ad39f245d4761a6ab2ea67ce600e58dc1f90"
+                "sha256": "3f24065f7d4b847951e9e098cbb2ad39f245d4761a6ab2ea67ce600e58dc1f90",
+                "url": "https://jcenter.bintray.com/org/apache/commons/commons-text/1.1/commons-text-1.1-sources.jar"
             },
             {
                 "coord": "org.apache.curator:curator-client:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar",
-                "directDependencies": [
-                    "com.google.guava:guava:18.0",
-                    "org.apache.zookeeper:zookeeper:3.5.8",
-                    "org.slf4j:slf4j-api:1.7.30"
-                ],
                 "dependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "org.slf4j:slf4j-api:1.7.30",
@@ -3685,22 +3684,22 @@
                     "io.netty:netty-codec:4.1.48.Final",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar",
+                "directDependencies": [
+                    "com.google.guava:guava:18.0",
+                    "org.apache.zookeeper:zookeeper:3.5.8",
+                    "org.slf4j:slf4j-api:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar",
                     "https://maven.google.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar"
                 ],
-                "sha256": "e1897adea52f7c7be9860fcc93f788682cc37a0bb92a8c44ef85a1dc9e566542"
+                "sha256": "e1897adea52f7c7be9860fcc93f788682cc37a0bb92a8c44ef85a1dc9e566542",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0.jar"
             },
             {
                 "coord": "org.apache.curator:curator-client:jar:sources:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar",
-                "directDependencies": [
-                    "com.google.guava:guava:jar:sources:18.0",
-                    "org.apache.zookeeper:zookeeper:jar:sources:3.5.8",
-                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
-                ],
                 "dependencies": [
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
@@ -3717,20 +3716,22 @@
                     "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar",
+                "directDependencies": [
+                    "com.google.guava:guava:jar:sources:18.0",
+                    "org.apache.zookeeper:zookeeper:jar:sources:3.5.8",
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar",
                     "https://maven.google.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar"
                 ],
-                "sha256": "edb3e57adba38bebebba10e662030a47f1c5f65b0903229fea851e50bb7106e4"
+                "sha256": "edb3e57adba38bebebba10e662030a47f1c5f65b0903229fea851e50bb7106e4",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-client/2.9.0/curator-client-2.9.0-sources.jar"
             },
             {
                 "coord": "org.apache.curator:curator-framework:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar",
-                "directDependencies": [
-                    "org.apache.curator:curator-client:2.9.0"
-                ],
                 "dependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "org.slf4j:slf4j-api:1.7.30",
@@ -3748,20 +3749,20 @@
                     "io.netty:netty-codec:4.1.48.Final",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar",
+                "directDependencies": [
+                    "org.apache.curator:curator-client:2.9.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar",
                     "https://maven.google.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar"
                 ],
-                "sha256": "c5fa90bb62eb7ae584dffb815954c617466c33a2dd271178c62e96a6b2613224"
+                "sha256": "c5fa90bb62eb7ae584dffb815954c617466c33a2dd271178c62e96a6b2613224",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0.jar"
             },
             {
                 "coord": "org.apache.curator:curator-framework:jar:sources:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.curator:curator-client:jar:sources:2.9.0"
-                ],
                 "dependencies": [
                     "org.apache.curator:curator-client:jar:sources:2.9.0",
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
@@ -3779,20 +3780,20 @@
                     "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar",
+                "directDependencies": [
+                    "org.apache.curator:curator-client:jar:sources:2.9.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar",
                     "https://maven.google.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar"
                 ],
-                "sha256": "daa3030204d3e855e6e077545929db544d1d11b41319330435ce20953514b5df"
+                "sha256": "daa3030204d3e855e6e077545929db544d1d11b41319330435ce20953514b5df",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-framework/2.9.0/curator-framework-2.9.0-sources.jar"
             },
             {
                 "coord": "org.apache.curator:curator-recipes:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar",
-                "directDependencies": [
-                    "org.apache.curator:curator-framework:2.9.0"
-                ],
                 "dependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "org.slf4j:slf4j-api:1.7.30",
@@ -3811,20 +3812,20 @@
                     "org.apache.curator:curator-framework:2.9.0",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar",
+                "directDependencies": [
+                    "org.apache.curator:curator-framework:2.9.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar",
                     "https://maven.google.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar"
                 ],
-                "sha256": "3f3a97f291e3f25565ea4c7206143c769e78dfacd5a67d8c3188ff4878fcc4e7"
+                "sha256": "3f3a97f291e3f25565ea4c7206143c769e78dfacd5a67d8c3188ff4878fcc4e7",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0.jar"
             },
             {
                 "coord": "org.apache.curator:curator-recipes:jar:sources:2.9.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.curator:curator-framework:jar:sources:2.9.0"
-                ],
                 "dependencies": [
                     "org.apache.curator:curator-client:jar:sources:2.9.0",
                     "io.netty:netty-buffer:jar:sources:4.1.48.Final",
@@ -3843,91 +3844,97 @@
                     "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar",
+                "directDependencies": [
+                    "org.apache.curator:curator-framework:jar:sources:2.9.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar",
                     "https://maven.google.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar"
                 ],
-                "sha256": "d953d9e660b90d3257a517e2830b69d68ac1b890e488134bd7419ea07e2e8c61"
+                "sha256": "d953d9e660b90d3257a517e2830b69d68ac1b890e488134bd7419ea07e2e8c61",
+                "url": "https://jcenter.bintray.com/org/apache/curator/curator-recipes/2.9.0/curator-recipes-2.9.0-sources.jar"
             },
             {
                 "coord": "org.apache.distributedlog:distributedlog-core-shaded:4.7.3",
-                "file": "v1/https/jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "commons-logging:commons-logging:1.2",
                     "org.slf4j:slf4j-api:1.7.30",
+                    "commons-lang:commons-lang:2.6",
                     "jline:jline:0.9.94",
+                    "junit:junit:4.12",
+                    "org.hamcrest:hamcrest-core:1.3",
                     "commons-configuration:commons-configuration:1.10",
                     "io.netty:netty-tcnative-boringssl-static:2.0.7.Final",
                     "com.beust:jcommander:1.48",
                     "org.apache.yetus:audience-annotations:0.5.0",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "dependencies": [
-                    "commons-logging:commons-logging:1.2",
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
-                    "commons-lang:commons-lang:2.6",
                     "jline:jline:0.9.94",
-                    "junit:junit:4.12",
-                    "org.hamcrest:hamcrest-core:1.3",
                     "commons-configuration:commons-configuration:1.10",
                     "io.netty:netty-tcnative-boringssl-static:2.0.7.Final",
                     "com.beust:jcommander:1.48",
                     "org.apache.yetus:audience-annotations:0.5.0",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar",
                     "https://maven.google.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar",
                     "https://repo1.maven.org/maven2/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar"
                 ],
-                "sha256": "6440765a50ecb87259692c430b03ba7657514c04feacdfcef69fae5ec8dec356"
+                "sha256": "6440765a50ecb87259692c430b03ba7657514c04feacdfcef69fae5ec8dec356",
+                "url": "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3.jar"
             },
             {
                 "coord": "org.apache.distributedlog:distributedlog-core-shaded:jar:sources:4.7.3",
-                "file": "v1/https/jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "jline:jline:jar:sources:0.9.94",
                     "io.netty:netty-tcnative-boringssl-static:jar:sources:2.0.7.Final",
                     "com.beust:jcommander:jar:sources:1.48",
+                    "commons-lang:commons-lang:jar:sources:2.6",
                     "commons-configuration:commons-configuration:jar:sources:1.10",
+                    "commons-logging:commons-logging:jar:sources:1.2",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "io.netty:netty-all:jar:sources:4.1.22.Final",
-                    "org.apache.yetus:audience-annotations:jar:sources:0.5.0"
+                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
+                    "org.apache.yetus:audience-annotations:jar:sources:0.5.0",
+                    "junit:junit:jar:sources:4.12"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "jline:jline:jar:sources:0.9.94",
                     "io.netty:netty-tcnative-boringssl-static:jar:sources:2.0.7.Final",
                     "com.beust:jcommander:jar:sources:1.48",
-                    "commons-lang:commons-lang:jar:sources:2.6",
                     "commons-configuration:commons-configuration:jar:sources:1.10",
-                    "commons-logging:commons-logging:jar:sources:1.2",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "io.netty:netty-all:jar:sources:4.1.22.Final",
-                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
-                    "org.apache.yetus:audience-annotations:jar:sources:0.5.0",
-                    "junit:junit:jar:sources:4.12"
+                    "org.apache.yetus:audience-annotations:jar:sources:0.5.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar",
                     "https://maven.google.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar"
                 ],
-                "sha256": "5122b156f4928615989c53a0f1ca828612c1a036c949d96a0ef561dc4e22dbd0"
+                "sha256": "5122b156f4928615989c53a0f1ca828612c1a036c949d96a0ef561dc4e22dbd0",
+                "url": "https://jcenter.bintray.com/org/apache/distributedlog/distributedlog-core-shaded/4.7.3/distributedlog-core-shaded-4.7.3-sources.jar"
             },
             {
                 "coord": "org.apache.hadoop:hadoop-core:0.20.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.eclipse.jdt:core:3.1.1",
                     "xmlenc:xmlenc:0.52",
                     "hsqldb:hsqldb:1.8.0.10",
+                    "commons-logging:commons-logging:1.2",
                     "org.mortbay.jetty:jsp-api-2.1:6.1.14",
                     "org.mortbay.jetty:jetty-util:6.1.14",
+                    "ant:ant:1.6.5",
                     "junit:junit:4.12",
                     "commons-codec:commons-codec:1.11",
+                    "org.hamcrest:hamcrest-core:1.3",
                     "commons-el:commons-el:1.0",
                     "org.mortbay.jetty:jetty:6.1.14",
                     "commons-cli:commons-cli:1.3.1",
@@ -3941,17 +3948,14 @@
                     "commons-net:commons-net:1.4.1",
                     "org.mortbay.jetty:jsp-2.1:6.1.14"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.eclipse.jdt:core:3.1.1",
                     "xmlenc:xmlenc:0.52",
                     "hsqldb:hsqldb:1.8.0.10",
-                    "commons-logging:commons-logging:1.2",
                     "org.mortbay.jetty:jsp-api-2.1:6.1.14",
                     "org.mortbay.jetty:jetty-util:6.1.14",
-                    "ant:ant:1.6.5",
                     "junit:junit:4.12",
                     "commons-codec:commons-codec:1.11",
-                    "org.hamcrest:hamcrest-core:1.3",
                     "commons-el:commons-el:1.0",
                     "org.mortbay.jetty:jetty:6.1.14",
                     "commons-cli:commons-cli:1.3.1",
@@ -3965,234 +3969,226 @@
                     "commons-net:commons-net:1.4.1",
                     "org.mortbay.jetty:jsp-2.1:6.1.14"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar",
                     "https://maven.google.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar",
                     "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar"
                 ],
-                "sha256": "3699d07e35cf58982d228bebaf3073ef4cf84e0f4f82dc91636118cab132e4f6"
+                "sha256": "3699d07e35cf58982d228bebaf3073ef4cf84e0f4f82dc91636118cab132e4f6",
+                "url": "https://jcenter.bintray.com/org/apache/hadoop/hadoop-core/0.20.2/hadoop-core-0.20.2.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpclient:4.5.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:1.11",
-                    "commons-logging:commons-logging:1.2",
-                    "org.apache.httpcomponents:httpcore:4.4.4"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "commons-codec:commons-codec:1.11"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
+                "directDependencies": [
+                    "commons-codec:commons-codec:1.11",
+                    "commons-logging:commons-logging:1.2",
+                    "org.apache.httpcomponents:httpcore:4.4.4"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar"
                 ],
-                "sha256": "0dffc621400d6c632f55787d996b8aeca36b30746a716e079a985f24d8074057"
+                "sha256": "0dffc621400d6c632f55787d996b8aeca36b30746a716e079a985f24d8074057",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpclient:jar:sources:4.5.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:jar:sources:1.11",
-                    "commons-logging:commons-logging:jar:sources:1.2",
-                    "org.apache.httpcomponents:httpcore:jar:sources:4.4.4"
-                ],
                 "dependencies": [
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "commons-codec:commons-codec:jar:sources:1.11",
                     "commons-logging:commons-logging:jar:sources:1.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar",
+                "directDependencies": [
+                    "commons-codec:commons-codec:jar:sources:1.11",
+                    "commons-logging:commons-logging:jar:sources:1.2",
+                    "org.apache.httpcomponents:httpcore:jar:sources:4.4.4"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar"
                 ],
-                "sha256": "1eb8d2b8b0b1528fbda8b1dbaf65e0412d29ce78cd37040b94b8fd1d58fef85a"
+                "sha256": "1eb8d2b8b0b1528fbda8b1dbaf65e0412d29ce78cd37040b94b8fd1d58fef85a",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-sources.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpclient:jar:tests:4.5.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar",
-                "directDependencies": [
-                    "commons-codec:commons-codec:1.11",
-                    "commons-logging:commons-logging:1.2",
-                    "org.apache.httpcomponents:httpcore:4.4.4"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging:1.2",
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "commons-codec:commons-codec:1.11"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar",
+                "directDependencies": [
+                    "commons-codec:commons-codec:1.11",
+                    "commons-logging:commons-logging:1.2",
+                    "org.apache.httpcomponents:httpcore:4.4.4"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar"
                 ],
-                "sha256": "d87c8dcfd609085090e5ced0249c5ac064c5d50bda29eb8633794fd91789d163"
+                "sha256": "d87c8dcfd609085090e5ced0249c5ac064c5d50bda29eb8633794fd91789d163",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2-tests.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpcore:4.4.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar"
                 ],
-                "sha256": "f7bc09dc8a7003822d109634ffd3845d579d12e725ae54673e323a7ce7f5e325"
+                "sha256": "f7bc09dc8a7003822d109634ffd3845d579d12e725ae54673e323a7ce7f5e325",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar"
                 ],
-                "sha256": "bf8a17189e48ae3964fb8ab79c32e6b87423229c38571024895d9c532bba6129"
+                "sha256": "bf8a17189e48ae3964fb8ab79c32e6b87423229c38571024895d9c532bba6129",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4-sources.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpmime:4.5.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar",
-                "directDependencies": [
-                    "org.apache.httpcomponents:httpclient:4.5.2"
-                ],
                 "dependencies": [
                     "org.apache.httpcomponents:httpclient:4.5.2",
                     "commons-logging:commons-logging:1.2",
                     "org.apache.httpcomponents:httpcore:4.4.4",
                     "commons-codec:commons-codec:1.11"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar",
+                "directDependencies": [
+                    "org.apache.httpcomponents:httpclient:4.5.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar"
                 ],
-                "sha256": "231a3f7e4962053db2be8461d5422e68fc458a3a7dd7d8ada803a348e21f8f07"
+                "sha256": "231a3f7e4962053db2be8461d5422e68fc458a3a7dd7d8ada803a348e21f8f07",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2.jar"
             },
             {
                 "coord": "org.apache.httpcomponents:httpmime:jar:sources:4.5.2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar",
-                "directDependencies": [
-                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
-                ],
                 "dependencies": [
                     "org.apache.httpcomponents:httpcore:jar:sources:4.4.4",
                     "commons-codec:commons-codec:jar:sources:1.11",
                     "commons-logging:commons-logging:jar:sources:1.2",
                     "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar",
+                "directDependencies": [
+                    "org.apache.httpcomponents:httpclient:jar:sources:4.5.2"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar",
                     "https://maven.google.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar"
                 ],
-                "sha256": "accfedcbe11b7eb496b70d0a19ee2b7cb26066c98450f51de9a6866b82a4ae4b"
+                "sha256": "accfedcbe11b7eb496b70d0a19ee2b7cb26066c98450f51de9a6866b82a4ae4b",
+                "url": "https://jcenter.bintray.com/org/apache/httpcomponents/httpmime/4.5.2/httpmime-4.5.2-sources.jar"
             },
             {
                 "coord": "org.apache.kafka:kafka-clients:2.2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.github.luben:zstd-jni:1.3.8-1",
                     "org.lz4:lz4-java:1.5.0",
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.xerial.snappy:snappy-java:1.1.7.2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.github.luben:zstd-jni:1.3.8-1",
                     "org.lz4:lz4-java:1.5.0",
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.xerial.snappy:snappy-java:1.1.7.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar",
                     "https://maven.google.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar"
                 ],
-                "sha256": "cc819c61aeb40906d38d862d2c645bc960ac3274eb57b89ba2153b60e562f2a2"
+                "sha256": "cc819c61aeb40906d38d862d2c645bc960ac3274eb57b89ba2153b60e562f2a2",
+                "url": "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0.jar"
             },
             {
                 "coord": "org.apache.kafka:kafka-clients:jar:sources:2.2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar",
-                "directDependencies": [
-                    "com.github.luben:zstd-jni:jar:sources:1.3.8-1",
+                "dependencies": [
                     "org.lz4:lz4-java:jar:sources:1.5.0",
+                    "com.github.luben:zstd-jni:jar:sources:1.3.8-1",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "org.xerial.snappy:snappy-java:jar:sources:1.1.7.2"
                 ],
-                "dependencies": [
-                    "org.lz4:lz4-java:jar:sources:1.5.0",
+                "directDependencies": [
                     "com.github.luben:zstd-jni:jar:sources:1.3.8-1",
+                    "org.lz4:lz4-java:jar:sources:1.5.0",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "org.xerial.snappy:snappy-java:jar:sources:1.1.7.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar",
                     "https://maven.google.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar"
                 ],
-                "sha256": "6337e5677ba1bb683d8bb386b86a6eb542de90a8eb345c309b9ac1990147fd86"
+                "sha256": "6337e5677ba1bb683d8bb386b86a6eb542de90a8eb345c309b9ac1990147fd86",
+                "url": "https://jcenter.bintray.com/org/apache/kafka/kafka-clients/2.2.0/kafka-clients-2.2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven.reporting:maven-reporting-api:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "doxia:doxia-sink-api:1.0-alpha-4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "doxia:doxia-sink-api:1.0-alpha-4"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar",
                     "https://maven.google.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar"
                 ],
-                "sha256": "b9caa4a383c9477b0fdd876396aca81afb98ddc661c1505dc5ed814765716c4a"
+                "sha256": "b9caa4a383c9477b0fdd876396aca81afb98ddc661c1505dc5ed814765716c4a",
+                "url": "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0.jar"
             },
             {
                 "coord": "org.apache.maven.reporting:maven-reporting-api:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "doxia:doxia-sink-api:jar:sources:1.0-alpha-4"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "doxia:doxia-sink-api:jar:sources:1.0-alpha-4"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar"
                 ],
-                "sha256": "28345ac9467d0f963f84a0eba8256149495ce385a7e229237be9e4adbfa0551e"
+                "sha256": "28345ac9467d0f963f84a0eba8256149495ce385a7e229237be9e4adbfa0551e",
+                "url": "https://jcenter.bintray.com/org/apache/maven/reporting/maven-reporting-api/2.0/maven-reporting-api-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven.shared:maven-common-artifact-filters:1.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:2.0.8",
-                    "org.codehaus.plexus:plexus-utils:3.0.8",
-                    "org.apache.maven:maven-project:2.0.8",
-                    "org.apache.maven:maven-plugin-api:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.apache.maven:maven-artifact:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.maven:maven-repository-metadata:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
@@ -4214,25 +4210,25 @@
                     "org.codehaus.plexus:plexus-classworlds:2.2.2",
                     "org.apache.maven:maven-settings:2.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:2.0.8",
+                    "org.codehaus.plexus:plexus-utils:3.0.8",
+                    "org.apache.maven:maven-project:2.0.8",
+                    "org.apache.maven:maven-plugin-api:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.apache.maven:maven-artifact:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar",
                     "https://maven.google.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar"
                 ],
-                "sha256": "5a769ea4c7530fb53c6b4b979a0f822e4c07770bc696838280abd1f9467abe08"
+                "sha256": "5a769ea4c7530fb53c6b4b979a0f822e4c07770bc696838280abd1f9467abe08",
+                "url": "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4.jar"
             },
             {
                 "coord": "org.apache.maven.shared:maven-common-artifact-filters:jar:sources:1.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
-                    "org.apache.maven:maven-project:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.apache.maven:maven-model:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
-                    "org.apache.maven:maven-artifact:jar:sources:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.maven:maven-settings:jar:sources:2.0.8",
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
@@ -4254,139 +4250,147 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
+                    "org.apache.maven:maven-project:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.apache.maven:maven-model:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
+                    "org.apache.maven:maven-artifact:jar:sources:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar",
                     "https://maven.google.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar"
                 ],
-                "sha256": "bacfaa0ea356c45aa674f1c722f659c5618d3c3e2abfc0ebf333d7ce558d15df"
+                "sha256": "bacfaa0ea356c45aa674f1c722f659c5618d3c3e2abfc0ebf333d7ce558d15df",
+                "url": "https://jcenter.bintray.com/org/apache/maven/shared/maven-common-artifact-filters/1.4/maven-common-artifact-filters-1.4-sources.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-file:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar"
                 ],
-                "sha256": "af086ef7698b0e81f2ea54fc5f704d01e9f693d757143a04a17c4958d4da8107"
+                "sha256": "af086ef7698b0e81f2ea54fc5f704d01e9f693d757143a04a17c4958d4da8107",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-file:jar:sources:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar"
                 ],
-                "sha256": "7cda8d775a77815ded1263f8185fdd1212e029f28404d6868813884a2345d48f"
+                "sha256": "7cda8d775a77815ded1263f8185fdd1212e029f28404d6868813884a2345d48f",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-file/1.0-alpha-5/wagon-file-1.0-alpha-5-sources.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-http-lightweight:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar"
                 ],
-                "sha256": "eefb87b65dd39731447d786ce2b16c403a192d2869c14fdd4da18892610aeeb2"
+                "sha256": "eefb87b65dd39731447d786ce2b16c403a192d2869c14fdd4da18892610aeeb2",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-http-lightweight:jar:sources:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar"
                 ],
-                "sha256": "388fee35c0429620f80001081cd3f70e255f3866bbb01b22ac1616933fa83d73"
+                "sha256": "388fee35c0429620f80001081cd3f70e255f3866bbb01b22ac1616933fa83d73",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-http-lightweight/1.0-alpha-5/wagon-http-lightweight-1.0-alpha-5-sources.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
                 "exclusions": [
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar"
                 ],
-                "sha256": "0ba6040074d1e193580bae9314392940f5ecd81e3b6d3b3381050360dce033ad"
+                "sha256": "0ba6040074d1e193580bae9314392940f5ecd81e3b6d3b3381050360dce033ad",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar"
                 ],
-                "sha256": "b193d3dbe73010677e2d465b57bf22b43067a10e61f0e8bdab4cc537eaacec06"
+                "sha256": "b193d3dbe73010677e2d465b57bf22b43067a10e61f0e8bdab4cc537eaacec06",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-provider-api/1.0-beta-2/wagon-provider-api-1.0-beta-2-sources.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-ssh:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "com.jcraft:jsch:0.1.23",
                     "org.codehaus.plexus:plexus-interactivity-api:1.0-alpha-4",
                     "classworlds:classworlds:1.1-alpha-2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "com.jcraft:jsch:0.1.23",
@@ -4397,25 +4401,25 @@
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar"
                 ],
-                "sha256": "45ed6921b5056219840128bdd8b628efee187957ab70314e35e0478f418c67d3"
+                "sha256": "45ed6921b5056219840128bdd8b628efee187957ab70314e35e0478f418c67d3",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5.jar"
             },
             {
                 "coord": "org.apache.maven.wagon:wagon-ssh:jar:sources:1.0-alpha-5",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "classworlds:classworlds:jar:sources:1.1-alpha-2",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.codehaus.plexus:plexus-interactivity-api:jar:sources:1.0-alpha-4",
                     "com.jcraft:jsch:jar:sources:0.1.23"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "classworlds:classworlds:jar:sources:1.1-alpha-2",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
@@ -4426,24 +4430,17 @@
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar",
                     "https://maven.google.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar"
                 ],
-                "sha256": "408a9ef688b312860deef7e57172d1b1a10bdddb6d895409d3ea4686680c0f1e"
+                "sha256": "408a9ef688b312860deef7e57172d1b1a10bdddb6d895409d3ea4686680c0f1e",
+                "url": "https://jcenter.bintray.com/org/apache/maven/wagon/wagon-ssh/1.0-alpha-5/wagon-ssh-1.0-alpha-5-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-artifact-manager:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-repository-metadata:2.0.8",
-                    "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
-                    "org.codehaus.plexus:plexus-utils:3.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.apache.maven:maven-artifact:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.maven:maven-repository-metadata:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
@@ -4458,24 +4455,24 @@
                     "org.apache.maven:maven-artifact:2.0.8",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-repository-metadata:2.0.8",
+                    "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
+                    "org.codehaus.plexus:plexus-utils:3.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.apache.maven:maven-artifact:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar"
                 ],
-                "sha256": "c257564b252dc69ff3f3603971164fcb387adbd000818e72c959e6be7b6319cd"
+                "sha256": "c257564b252dc69ff3f3603971164fcb387adbd000818e72c959e6be7b6319cd",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-artifact-manager:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
-                    "org.apache.maven:maven-artifact:jar:sources:2.0.8",
-                    "org.apache.maven:maven-repository-metadata:jar:sources:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -4490,128 +4487,140 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
+                    "org.apache.maven:maven-artifact:jar:sources:2.0.8",
+                    "org.apache.maven:maven-repository-metadata:jar:sources:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar"
                 ],
-                "sha256": "4779c14594f4b50c1b4fd46cc2ac3f4b95974fa96b9e4cf4d556540fb75a697d"
+                "sha256": "4779c14594f4b50c1b4fd46cc2ac3f4b95974fa96b9e4cf4d556540fb75a697d",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact-manager/2.0.8/maven-artifact-manager-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-artifact:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar"
                 ],
-                "sha256": "5cf23417cdee6a8e1eb6b9c015c8feea62cedb7dceb3ad098e6869569fcfd1c0"
+                "sha256": "5cf23417cdee6a8e1eb6b9c015c8feea62cedb7dceb3ad098e6869569fcfd1c0",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-artifact:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar"
                 ],
-                "sha256": "b4f95bf1d3ab23b2b6943630f1768dd82f3888d520b752b1c32a40e44541b97e"
+                "sha256": "b4f95bf1d3ab23b2b6943630f1768dd82f3888d520b752b1c32a40e44541b97e",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-artifact/2.0.8/maven-artifact-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-core:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.maven:maven-plugin-descriptor:2.0",
                     "org.apache.maven.reporting:maven-reporting-api:2.0",
                     "org.apache.maven:maven-repository-metadata:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
+                    "commons-logging:commons-logging-api:1.1",
                     "org.apache.maven:maven-model:2.0.8",
                     "org.apache.maven:maven-artifact-manager:2.0.8",
+                    "junit:junit:4.12",
+                    "log4j:log4j:1.2.17",
+                    "org.hamcrest:hamcrest-core:1.3",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "org.apache.maven:maven-project:2.0.8",
                     "org.apache.maven:maven-profile:2.0.8",
+                    "com.jcraft:jsch:0.1.23",
                     "org.apache.maven:maven-plugin-api:2.0.8",
                     "org.apache.maven:maven-error-diagnostics:2.0",
+                    "doxia:doxia-sink-api:1.0-alpha-4",
                     "org.codehaus.plexus:plexus-container-default:1.5.5",
                     "commons-cli:commons-cli:1.3.1",
                     "org.apache.maven:maven-plugin-registry:2.0.8",
                     "org.codehaus.plexus:plexus-interactivity-api:1.0-alpha-4",
+                    "com.google.collections:google-collections:1.0",
                     "org.apache.maven:maven-monitor:2.0",
+                    "org.apache.xbean:xbean-reflect:3.4",
                     "org.apache.maven:maven-artifact:2.0.8",
                     "org.apache.maven.wagon:wagon-http-lightweight:1.0-alpha-5",
                     "org.apache.maven.wagon:wagon-file:1.0-alpha-5",
+                    "classworlds:classworlds:1.1-alpha-2",
                     "org.apache.maven:maven-plugin-parameter-documenter:2.0",
+                    "org.codehaus.plexus:plexus-classworlds:2.2.2",
                     "org.apache.maven.wagon:wagon-ssh:1.0-alpha-5",
                     "org.apache.maven:maven-settings:2.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.maven:maven-plugin-descriptor:2.0",
                     "org.apache.maven.reporting:maven-reporting-api:2.0",
                     "org.apache.maven:maven-repository-metadata:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
-                    "commons-logging:commons-logging-api:1.1",
                     "org.apache.maven:maven-model:2.0.8",
                     "org.apache.maven:maven-artifact-manager:2.0.8",
-                    "junit:junit:4.12",
-                    "log4j:log4j:1.2.17",
-                    "org.hamcrest:hamcrest-core:1.3",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "org.apache.maven:maven-project:2.0.8",
                     "org.apache.maven:maven-profile:2.0.8",
-                    "com.jcraft:jsch:0.1.23",
                     "org.apache.maven:maven-plugin-api:2.0.8",
                     "org.apache.maven:maven-error-diagnostics:2.0",
-                    "doxia:doxia-sink-api:1.0-alpha-4",
                     "org.codehaus.plexus:plexus-container-default:1.5.5",
                     "commons-cli:commons-cli:1.3.1",
                     "org.apache.maven:maven-plugin-registry:2.0.8",
                     "org.codehaus.plexus:plexus-interactivity-api:1.0-alpha-4",
-                    "com.google.collections:google-collections:1.0",
                     "org.apache.maven:maven-monitor:2.0",
-                    "org.apache.xbean:xbean-reflect:3.4",
                     "org.apache.maven:maven-artifact:2.0.8",
                     "org.apache.maven.wagon:wagon-http-lightweight:1.0-alpha-5",
                     "org.apache.maven.wagon:wagon-file:1.0-alpha-5",
-                    "classworlds:classworlds:1.1-alpha-2",
                     "org.apache.maven:maven-plugin-parameter-documenter:2.0",
-                    "org.codehaus.plexus:plexus-classworlds:2.2.2",
                     "org.apache.maven.wagon:wagon-ssh:1.0-alpha-5",
                     "org.apache.maven:maven-settings:2.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar",
                     "https://maven.google.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-core/2.0/maven-core-2.0.jar"
                 ],
-                "sha256": "b4dce584563fafa05a0f987b2ce237f195809549abcf454ee61b7269c514c65a"
+                "sha256": "b4dce584563fafa05a0f987b2ce237f195809549abcf454ee61b7269c514c65a",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0.jar"
             },
             {
                 "coord": "org.apache.maven:maven-core:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.maven:maven-settings:jar:sources:2.0.8",
+                    "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "org.apache.maven:maven-profile:jar:sources:2.0.8",
                     "org.apache.maven:maven-monitor:jar:sources:2.0",
+                    "commons-logging:commons-logging-api:jar:sources:1.1",
+                    "com.google.collections:google-collections:jar:sources:1.0",
                     "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
                     "org.apache.maven:maven-artifact-manager:jar:sources:2.0.8",
+                    "classworlds:classworlds:jar:sources:1.1-alpha-2",
                     "org.apache.maven:maven-project:jar:sources:2.0.8",
                     "org.apache.maven.wagon:wagon-ssh:jar:sources:1.0-alpha-5",
+                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
                     "org.apache.maven:maven-plugin-registry:jar:sources:2.0.8",
                     "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
                     "org.apache.maven.wagon:wagon-file:jar:sources:1.0-alpha-5",
@@ -4619,28 +4628,28 @@
                     "org.apache.maven:maven-model:jar:sources:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
                     "org.apache.maven:maven-error-diagnostics:jar:sources:2.0",
+                    "doxia:doxia-sink-api:jar:sources:1.0-alpha-4",
                     "org.apache.maven.reporting:maven-reporting-api:jar:sources:2.0",
                     "org.apache.maven:maven-plugin-descriptor:jar:sources:2.0",
+                    "log4j:log4j:jar:sources:1.2.17",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.apache.maven:maven-artifact:jar:sources:2.0.8",
                     "org.apache.maven:maven-repository-metadata:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
+                    "junit:junit:jar:sources:4.12",
                     "org.apache.maven:maven-plugin-parameter-documenter:jar:sources:2.0",
                     "org.codehaus.plexus:plexus-interactivity-api:jar:sources:1.0-alpha-4",
+                    "com.jcraft:jsch:jar:sources:0.1.23",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.maven:maven-settings:jar:sources:2.0.8",
-                    "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "org.apache.maven:maven-profile:jar:sources:2.0.8",
                     "org.apache.maven:maven-monitor:jar:sources:2.0",
-                    "commons-logging:commons-logging-api:jar:sources:1.1",
-                    "com.google.collections:google-collections:jar:sources:1.0",
                     "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
                     "org.apache.maven:maven-artifact-manager:jar:sources:2.0.8",
-                    "classworlds:classworlds:jar:sources:1.1-alpha-2",
                     "org.apache.maven:maven-project:jar:sources:2.0.8",
                     "org.apache.maven.wagon:wagon-ssh:jar:sources:1.0-alpha-5",
-                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
                     "org.apache.maven:maven-plugin-registry:jar:sources:2.0.8",
                     "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
                     "org.apache.maven.wagon:wagon-file:jar:sources:1.0-alpha-5",
@@ -4648,34 +4657,26 @@
                     "org.apache.maven:maven-model:jar:sources:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:jar:sources:1.0-beta-2",
                     "org.apache.maven:maven-error-diagnostics:jar:sources:2.0",
-                    "doxia:doxia-sink-api:jar:sources:1.0-alpha-4",
                     "org.apache.maven.reporting:maven-reporting-api:jar:sources:2.0",
                     "org.apache.maven:maven-plugin-descriptor:jar:sources:2.0",
-                    "log4j:log4j:jar:sources:1.2.17",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.apache.maven:maven-artifact:jar:sources:2.0.8",
                     "org.apache.maven:maven-repository-metadata:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
-                    "junit:junit:jar:sources:4.12",
                     "org.apache.maven:maven-plugin-parameter-documenter:jar:sources:2.0",
                     "org.codehaus.plexus:plexus-interactivity-api:jar:sources:1.0-alpha-4",
-                    "com.jcraft:jsch:jar:sources:0.1.23",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar"
                 ],
-                "sha256": "90dae00fbee334390470b93f1bc0784736c6a2d79c713eaea918b47844664384"
+                "sha256": "90dae00fbee334390470b93f1bc0784736c6a2d79c713eaea918b47844664384",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-core/2.0/maven-core-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-error-diagnostics:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:1.5.5"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "junit:junit:4.12",
@@ -4687,20 +4688,20 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:1.5.5"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar",
                     "https://maven.google.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar"
                 ],
-                "sha256": "9e18532942abc78c13bd4debc76f4646ceeb80f11968d7a9e017e0b1b550524f"
+                "sha256": "9e18532942abc78c13bd4debc76f4646ceeb80f11968d7a9e017e0b1b550524f",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0.jar"
             },
             {
                 "coord": "org.apache.maven:maven-error-diagnostics:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -4712,108 +4713,106 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar"
                 ],
-                "sha256": "b166fbf5b2d4add7bfab0d970b4062f87278ac420cbd09827aa5818904e36550"
+                "sha256": "b166fbf5b2d4add7bfab0d970b4062f87278ac420cbd09827aa5818904e36550",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-error-diagnostics/2.0/maven-error-diagnostics-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-model:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar"
                 ],
-                "sha256": "51cde4f45d74720eaf567444d88c9ffdfc6896dd58dcc459403f6613d8439255"
+                "sha256": "51cde4f45d74720eaf567444d88c9ffdfc6896dd58dcc459403f6613d8439255",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-model:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar"
                 ],
-                "sha256": "3597ca8d371d04eaadd411e302aa45a1ad66723342175e77f0d4b112437899c7"
+                "sha256": "3597ca8d371d04eaadd411e302aa45a1ad66723342175e77f0d4b112437899c7",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-model/2.0.8/maven-model-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-monitor:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar",
                     "https://maven.google.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar"
                 ],
-                "sha256": "d3e450c2f38e6656d80133647a6abe4d1642ce2695c843fcfc2f7afaf6fa9f7c"
+                "sha256": "d3e450c2f38e6656d80133647a6abe4d1642ce2695c843fcfc2f7afaf6fa9f7c",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0.jar"
             },
             {
                 "coord": "org.apache.maven:maven-monitor:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar"
                 ],
-                "sha256": "c99f6d54814207027c531d90322db4e4b80aac45e938ee96d6b3fed0d19d1d80"
+                "sha256": "c99f6d54814207027c531d90322db4e4b80aac45e938ee96d6b3fed0d19d1d80",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-monitor/2.0/maven-monitor-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-api:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar"
                 ],
-                "sha256": "61050e4b31c7c887408ae52f2f48e2a5847c8923f0941729c3e3fe71e1baaed1"
+                "sha256": "61050e4b31c7c887408ae52f2f48e2a5847c8923f0941729c3e3fe71e1baaed1",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar"
                 ],
-                "sha256": "326b7a624ac2bf3acea6c8dd5bec6c4253714c6783b4ec62121b6e00f71500f4"
+                "sha256": "326b7a624ac2bf3acea6c8dd5bec6c4253714c6783b4ec62121b6e00f71500f4",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-api/2.0.8/maven-plugin-api-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-descriptor:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-artifact:2.0.8",
-                    "org.apache.maven:maven-plugin-api:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "junit:junit:4.12",
@@ -4827,22 +4826,22 @@
                     "org.apache.maven:maven-artifact:2.0.8",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-artifact:2.0.8",
+                    "org.apache.maven:maven-plugin-api:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar"
                 ],
-                "sha256": "b9fa0f3b72a161bf7a44a4eccaf03e5e54818d75a731a49cabab2ed6c2a5d36c"
+                "sha256": "b9fa0f3b72a161bf7a44a4eccaf03e5e54818d75a731a49cabab2ed6c2a5d36c",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-descriptor:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-artifact:jar:sources:2.0.8",
-                    "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -4856,21 +4855,22 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar",
-                "mirror_urls": [
+                "directDependencies": [
+                    "org.apache.maven:maven-artifact:jar:sources:2.0.8",
+                    "org.apache.maven:maven-plugin-api:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar",
+                "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar"
                 ],
-                "sha256": "26d9cff793efc4059b0d831a03597b6f6f75fcf8e0090e1a6b52efb0d369e138"
+                "sha256": "26d9cff793efc4059b0d831a03597b6f6f75fcf8e0090e1a6b52efb0d369e138",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-descriptor/2.0/maven-plugin-descriptor-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-parameter-documenter:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:3.0.8"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "junit:junit:4.12",
@@ -4882,21 +4882,21 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar"
                 ],
-                "sha256": "2189475025f392dce067258f5aa967653f409df0db0b11e27ce07ebd3d935c6b"
+                "sha256": "2189475025f392dce067258f5aa967653f409df0db0b11e27ce07ebd3d935c6b",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-parameter-documenter:jar:sources:2.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -4908,21 +4908,21 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar"
                 ],
-                "sha256": "50ca906177a10c1f44bb9f755c35a92af013d66f05c518fe45b5c9fb40d22bb2"
+                "sha256": "50ca906177a10c1f44bb9f755c35a92af013d66f05c518fe45b5c9fb40d22bb2",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-parameter-documenter/2.0/maven-plugin-parameter-documenter-2.0-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-registry:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:3.0.8"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "junit:junit:4.12",
@@ -4934,21 +4934,21 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar"
                 ],
-                "sha256": "a7e77626ec4c8382a4c11ba808684402582693df00f0c39f7c0fc02cd1ece9ab"
+                "sha256": "a7e77626ec4c8382a4c11ba808684402582693df00f0c39f7c0fc02cd1ece9ab",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-plugin-registry:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar",
-                "directDependencies": [
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -4960,22 +4960,21 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar",
+                "directDependencies": [
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar"
                 ],
-                "sha256": "0944967176ff5295765db11c3e593ed0368c1b70ddbd82019225592654503c2d"
+                "sha256": "0944967176ff5295765db11c3e593ed0368c1b70ddbd82019225592654503c2d",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-plugin-registry/2.0.8/maven-plugin-registry-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-profile:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:3.0.8"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "org.apache.maven:maven-model:2.0.8",
@@ -4988,22 +4987,22 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar"
                 ],
-                "sha256": "bd566c2fdb896e3dd157dc3e49b3c10f93250daaa6462af3cd42ad5b4aeda0a3"
+                "sha256": "bd566c2fdb896e3dd157dc3e49b3c10f93250daaa6462af3cd42ad5b4aeda0a3",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-profile:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -5016,27 +5015,22 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar"
                 ],
-                "sha256": "80f4392fc69be3cce5bafce69850304797a1a9c8ba66c8080ebb350a52493b95"
+                "sha256": "80f4392fc69be3cce5bafce69850304797a1a9c8ba66c8080ebb350a52493b95",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-profile/2.0.8/maven-profile-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-project:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:2.0.8",
-                    "org.apache.maven:maven-artifact-manager:2.0.8",
-                    "org.codehaus.plexus:plexus-utils:3.0.8",
-                    "org.apache.maven:maven-profile:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.apache.maven:maven-plugin-registry:2.0.8",
-                    "org.apache.maven:maven-artifact:2.0.8",
-                    "org.apache.maven:maven-settings:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.maven:maven-repository-metadata:2.0.8",
                     "org.apache.maven.wagon:wagon-provider-api:1.0-beta-2",
@@ -5056,27 +5050,27 @@
                     "org.codehaus.plexus:plexus-classworlds:2.2.2",
                     "org.apache.maven:maven-settings:2.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:2.0.8",
+                    "org.apache.maven:maven-artifact-manager:2.0.8",
+                    "org.codehaus.plexus:plexus-utils:3.0.8",
+                    "org.apache.maven:maven-profile:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.apache.maven:maven-plugin-registry:2.0.8",
+                    "org.apache.maven:maven-artifact:2.0.8",
+                    "org.apache.maven:maven-settings:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar"
                 ],
-                "sha256": "46799ed8812c96f1e651958cbd864155cc8a9ae8d4ebe2392b845d232a910d31"
+                "sha256": "46799ed8812c96f1e651958cbd864155cc8a9ae8d4ebe2392b845d232a910d31",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-project:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-settings:jar:sources:2.0.8",
-                    "org.apache.maven:maven-profile:jar:sources:2.0.8",
-                    "org.apache.maven:maven-artifact-manager:jar:sources:2.0.8",
-                    "org.apache.maven:maven-plugin-registry:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.apache.maven:maven-model:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
-                    "org.apache.maven:maven-artifact:jar:sources:2.0.8"
-                ],
                 "dependencies": [
                     "org.apache.maven:maven-settings:jar:sources:2.0.8",
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
@@ -5096,56 +5090,61 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-settings:jar:sources:2.0.8",
+                    "org.apache.maven:maven-profile:jar:sources:2.0.8",
+                    "org.apache.maven:maven-artifact-manager:jar:sources:2.0.8",
+                    "org.apache.maven:maven-plugin-registry:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.apache.maven:maven-model:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
+                    "org.apache.maven:maven-artifact:jar:sources:2.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar"
                 ],
-                "sha256": "88c50e2d015a7d77a9b55af5e711f506b00b9a7334b5b5e237fc26f64f9d2018"
+                "sha256": "88c50e2d015a7d77a9b55af5e711f506b00b9a7334b5b5e237fc26f64f9d2018",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-project/2.0.8/maven-project-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-repository-metadata:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar"
                 ],
-                "sha256": "aff8473e802e4e1c226a777a198f72fbdf7ef36f6f972df6de763b767f652ee1"
+                "sha256": "aff8473e802e4e1c226a777a198f72fbdf7ef36f6f972df6de763b767f652ee1",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-repository-metadata:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar"
                 ],
-                "sha256": "67bd67b11fce2dd66d08a49ec28d6098c71707c46d71cf011ba8c1a754988cc1"
+                "sha256": "67bd67b11fce2dd66d08a49ec28d6098c71707c46d71cf011ba8c1a754988cc1",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-repository-metadata/2.0.8/maven-repository-metadata-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.maven:maven-settings:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:3.0.8"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "org.apache.maven:maven-model:2.0.8",
@@ -5158,22 +5157,22 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar",
                     "https://maven.google.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar"
                 ],
-                "sha256": "e1873a36ea2debc0bb6210c3064faae91cf12108e9f8b9845e283022f58ecb46"
+                "sha256": "e1873a36ea2debc0bb6210c3064faae91cf12108e9f8b9845e283022f58ecb46",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8.jar"
             },
             {
                 "coord": "org.apache.maven:maven-settings:jar:sources:2.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar",
-                "directDependencies": [
-                    "org.apache.maven:maven-model:jar:sources:2.0.8",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -5186,101 +5185,98 @@
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar",
+                "directDependencies": [
+                    "org.apache.maven:maven-model:jar:sources:2.0.8",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar",
                     "https://maven.google.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar"
                 ],
-                "sha256": "c0a42707ef2620bb121c3fd561f5f390620fea862cd55c825e345aca86b51632"
+                "sha256": "c0a42707ef2620bb121c3fd561f5f390620fea862cd55c825e345aca86b51632",
+                "url": "https://jcenter.bintray.com/org/apache/maven/maven-settings/2.0.8/maven-settings-2.0.8-sources.jar"
             },
             {
                 "coord": "org.apache.mesos:mesos:0.22.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.google.protobuf:protobuf-java:3.4.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.protobuf:protobuf-java:3.4.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar",
                     "https://maven.google.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar"
                 ],
-                "sha256": "3bd5358c77995a1858db9173927757a68f762ba8e19430ffaf9b85301d726490"
+                "sha256": "3bd5358c77995a1858db9173927757a68f762ba8e19430ffaf9b85301d726490",
+                "url": "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0.jar"
             },
             {
                 "coord": "org.apache.mesos:mesos:jar:sources:0.22.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar",
                     "https://maven.google.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar"
                 ],
-                "sha256": "46dda0412d03c87451fa3d7093370ba98f03673e7a279d0c201cc395e149c61f"
+                "sha256": "46dda0412d03c87451fa3d7093370ba98f03673e7a279d0c201cc395e149c61f",
+                "url": "https://jcenter.bintray.com/org/apache/mesos/mesos/0.22.0/mesos-0.22.0-sources.jar"
             },
             {
                 "coord": "org.apache.pulsar:pulsar-checksum:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar",
-                "directDependencies": [
-                    "com.google.guava:guava:18.0",
-                    "io.netty:netty-all:4.1.22.Final",
-                    "org.slf4j:slf4j-api:1.7.30"
-                ],
                 "dependencies": [
                     "com.google.guava:guava:18.0",
                     "org.slf4j:slf4j-api:1.7.30",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar",
+                "directDependencies": [
+                    "com.google.guava:guava:18.0",
+                    "io.netty:netty-all:4.1.22.Final",
+                    "org.slf4j:slf4j-api:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar",
                     "https://maven.google.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar",
                     "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar"
                 ],
-                "sha256": "9631400c6ea9e2de4498e1a2fab606f6134b09fa53537a1f523851413cff7cc1"
+                "sha256": "9631400c6ea9e2de4498e1a2fab606f6134b09fa53537a1f523851413cff7cc1",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating.jar"
             },
             {
                 "coord": "org.apache.pulsar:pulsar-checksum:jar:sources:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar",
-                "directDependencies": [
-                    "com.google.guava:guava:jar:sources:18.0",
-                    "io.netty:netty-all:jar:sources:4.1.22.Final",
-                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
-                ],
                 "dependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "io.netty:netty-all:jar:sources:4.1.22.Final",
                     "com.google.guava:guava:jar:sources:18.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar",
+                "directDependencies": [
+                    "com.google.guava:guava:jar:sources:18.0",
+                    "io.netty:netty-all:jar:sources:4.1.22.Final",
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar",
                     "https://maven.google.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar"
                 ],
-                "sha256": "81d9a5ccb743501a1c8295a5774292caf2ba899db9c22f7e3b54852344f1f9b8"
+                "sha256": "81d9a5ccb743501a1c8295a5774292caf2ba899db9c22f7e3b54852344f1f9b8",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-checksum/1.19.0-incubating/pulsar-checksum-1.19.0-incubating-sources.jar"
             },
             {
-                "coord": "org.apache.pulsar:pulsar-client:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating.jar",
-                "directDependencies": [
-                    "org.slf4j:slf4j-api:1.7.30",
-                    "commons-codec:commons-codec:1.11",
-                    "org.apache.commons:commons-lang3:3.7",
-                    "org.apache.pulsar:pulsar-common:1.19.0-incubating",
-                    "com.yahoo.datasketches:sketches-core:0.6.0",
-                    "org.asynchttpclient:async-http-client:2.0.31"
-                ],
+                "coord": "org.apache.pulsar:pulsar-client:jar:shaded:1.19.0-incubating",
                 "dependencies": [
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "org.apache.pulsar:pulsar-checksum:1.19.0-incubating",
@@ -5303,25 +5299,25 @@
                     "io.netty:netty-all:4.1.22.Final",
                     "net.jpountz.lz4:lz4:1.3.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating.jar",
+                "directDependencies": [
+                    "org.slf4j:slf4j-api:1.7.30",
+                    "commons-codec:commons-codec:1.11",
+                    "org.apache.commons:commons-lang3:3.7",
+                    "org.apache.pulsar:pulsar-common:1.19.0-incubating",
+                    "com.yahoo.datasketches:sketches-core:0.6.0",
+                    "org.asynchttpclient:async-http-client:2.0.31"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-shaded.jar",
                 "mirror_urls": [
-                    "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating.jar",
-                    "https://maven.google.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating.jar",
-                    "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating.jar"
+                    "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-shaded.jar",
+                    "https://maven.google.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-shaded.jar",
+                    "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-shaded.jar"
                 ],
-                "sha256": "7175f2fcdef0dd30d3149f2e96ba55134fa490f36ae12795139a30f8d507dc46"
+                "sha256": "7f740effec8dbce3f16e4081ee08f3d16a6419899520495933facdc9c8d5f5c1",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-shaded.jar"
             },
             {
                 "coord": "org.apache.pulsar:pulsar-client:jar:sources:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar",
-                "directDependencies": [
-                    "org.asynchttpclient:async-http-client:jar:sources:2.0.31",
-                    "com.yahoo.datasketches:sketches-core:jar:sources:0.6.0",
-                    "org.slf4j:slf4j-api:jar:sources:1.7.30",
-                    "org.apache.pulsar:pulsar-common:jar:sources:1.19.0-incubating",
-                    "org.apache.commons:commons-lang3:jar:sources:3.7",
-                    "commons-codec:commons-codec:jar:sources:1.11"
-                ],
                 "dependencies": [
                     "org.asynchttpclient:netty-resolver-dns:jar:sources:2.0.31",
                     "net.jpountz.lz4:lz4:jar:sources:1.3.0",
@@ -5344,120 +5340,118 @@
                     "org.reactivestreams:reactive-streams:jar:sources:1.0.0",
                     "com.typesafe.netty:netty-reactive-streams:jar:sources:1.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar",
+                "directDependencies": [
+                    "org.asynchttpclient:async-http-client:jar:sources:2.0.31",
+                    "com.yahoo.datasketches:sketches-core:jar:sources:0.6.0",
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30",
+                    "org.apache.pulsar:pulsar-common:jar:sources:1.19.0-incubating",
+                    "org.apache.commons:commons-lang3:jar:sources:3.7",
+                    "commons-codec:commons-codec:jar:sources:1.11"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar",
                     "https://maven.google.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar"
                 ],
-                "sha256": "0df714022988814de6d02c6a8143565cc5e6e64d9a15cecdcb76745a40ca4c79"
+                "sha256": "0df714022988814de6d02c6a8143565cc5e6e64d9a15cecdcb76745a40ca4c79",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-client/1.19.0-incubating/pulsar-client-1.19.0-incubating-sources.jar"
             },
             {
                 "coord": "org.apache.pulsar:pulsar-common:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
                     "org.apache.pulsar:pulsar-checksum:1.19.0-incubating",
                     "org.slf4j:slf4j-api:1.7.30",
                     "com.google.protobuf:protobuf-java:3.4.0",
                     "com.google.guava:guava:18.0",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
                     "io.netty:netty-all:4.1.22.Final",
                     "net.jpountz.lz4:lz4:1.3.0"
                 ],
-                "dependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
+                "directDependencies": [
                     "org.apache.pulsar:pulsar-checksum:1.19.0-incubating",
                     "org.slf4j:slf4j-api:1.7.30",
                     "com.google.protobuf:protobuf-java:3.4.0",
                     "com.google.guava:guava:18.0",
                     "com.fasterxml.jackson.core:jackson-databind:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-core:2.8.8",
                     "io.netty:netty-all:4.1.22.Final",
                     "net.jpountz.lz4:lz4:1.3.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar",
                     "https://maven.google.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar",
                     "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar"
                 ],
-                "sha256": "4b21e04138d48403593f2010dacc63b14cffc6a9a785c7190608205ebd09352d"
+                "sha256": "4b21e04138d48403593f2010dacc63b14cffc6a9a785c7190608205ebd09352d",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating.jar"
             },
             {
                 "coord": "org.apache.pulsar:pulsar-common:jar:sources:1.19.0-incubating",
-                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "net.jpountz.lz4:lz4:jar:sources:1.3.0",
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "io.netty:netty-all:jar:sources:4.1.22.Final",
+                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.google.guava:guava:jar:sources:18.0",
                     "org.apache.pulsar:pulsar-checksum:jar:sources:1.19.0-incubating",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "net.jpountz.lz4:lz4:jar:sources:1.3.0",
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
                     "io.netty:netty-all:jar:sources:4.1.22.Final",
-                    "com.fasterxml.jackson.core:jackson-core:jar:sources:2.8.8",
                     "com.google.guava:guava:jar:sources:18.0",
                     "org.apache.pulsar:pulsar-checksum:jar:sources:1.19.0-incubating",
                     "com.fasterxml.jackson.core:jackson-databind:jar:sources:2.8.8",
                     "com.google.protobuf:protobuf-java:jar:sources:3.4.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar",
                     "https://maven.google.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar"
                 ],
-                "sha256": "7b990be2752b113b73a6ceb0dcc99f0c8e3d99ac90b54f889c6ccd2ab669ee39"
+                "sha256": "7b990be2752b113b73a6ceb0dcc99f0c8e3d99ac90b54f889c6ccd2ab669ee39",
+                "url": "https://jcenter.bintray.com/org/apache/pulsar/pulsar-common/1.19.0-incubating/pulsar-common-1.19.0-incubating-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-annotations:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar"
                 ],
-                "sha256": "0d40359495d66571b186a186835ce60a6863cd1e557ab82673db5fbe0604cdef"
+                "sha256": "0d40359495d66571b186a186835ce60a6863cd1e557ab82673db5fbe0604cdef",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-annotations:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar"
                 ],
-                "sha256": "b0f5bc0460c9c2c89e0c122b5e6c534bf25a17059e8097834649dfeadead54a1"
+                "sha256": "b0f5bc0460c9c2c89e0c122b5e6c534bf25a17059e8097834649dfeadead54a1",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-annotations/0.14.0/reef-annotations-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-common:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "org.apache.avro:avro:1.7.4",
+                    "org.ow2.asm:asm:5.0.4",
                     "org.apache.reef:wake:0.14.0",
-                    "com.google.code.findbugs:jsr305:3.0.2",
-                    "org.apache.reef:reef-annotations:0.14.0",
-                    "org.apache.reef:reef-utils:0.14.0",
-                    "org.apache.commons:commons-lang3:3.7",
-                    "org.apache.reef:tang:0.14.0",
-                    "net.jcip:jcip-annotations:1.0",
-                    "com.google.protobuf:protobuf-java:3.4.0"
-                ],
-                "dependencies": [
-                    "org.apache.avro:avro:1.7.4",
-                    "org.ow2.asm:asm:5.0.4",
-                    "org.apache.reef:wake:0.14.0",
-                    "commons-logging:commons-logging:1.2",
+                    "commons-logging:commons-logging:1.2",
                     "com.google.code.findbugs:jsr305:3.0.2",
                     "org.slf4j:slf4j-api:1.7.30",
                     "com.thoughtworks.paranamer:paranamer:2.3",
@@ -5478,27 +5472,27 @@
                     "org.codehaus.jackson:jackson-core-asl:1.8.8",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar",
+                "directDependencies": [
+                    "org.apache.reef:wake:0.14.0",
+                    "com.google.code.findbugs:jsr305:3.0.2",
+                    "org.apache.reef:reef-annotations:0.14.0",
+                    "org.apache.reef:reef-utils:0.14.0",
+                    "org.apache.commons:commons-lang3:3.7",
+                    "org.apache.reef:tang:0.14.0",
+                    "net.jcip:jcip-annotations:1.0",
+                    "com.google.protobuf:protobuf-java:3.4.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar"
                 ],
-                "sha256": "d39fbf28b9a64f80e538235e4e190a356ca1756c77f0b973c78b2fb3da6d3ac8"
+                "sha256": "d39fbf28b9a64f80e538235e4e190a356ca1756c77f0b973c78b2fb3da6d3ac8",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-common:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar",
-                "directDependencies": [
-                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
-                    "net.jcip:jcip-annotations:jar:sources:1.0",
-                    "org.apache.reef:reef-annotations:jar:sources:0.14.0",
-                    "org.apache.reef:reef-utils:jar:sources:0.14.0",
-                    "org.apache.commons:commons-lang3:jar:sources:3.7",
-                    "org.apache.reef:wake:jar:sources:0.14.0",
-                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
-                    "org.apache.reef:tang:jar:sources:0.14.0"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
@@ -5524,20 +5518,27 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar",
+                "directDependencies": [
+                    "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
+                    "net.jcip:jcip-annotations:jar:sources:1.0",
+                    "org.apache.reef:reef-annotations:jar:sources:0.14.0",
+                    "org.apache.reef:reef-utils:jar:sources:0.14.0",
+                    "org.apache.commons:commons-lang3:jar:sources:3.7",
+                    "org.apache.reef:wake:jar:sources:0.14.0",
+                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
+                    "org.apache.reef:tang:jar:sources:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar"
                 ],
-                "sha256": "7d5037210a6287787bbad0cf1312f37e0299dd33749bd3cc5dc08dffd3b1ed6f"
+                "sha256": "7d5037210a6287787bbad0cf1312f37e0299dd33749bd3cc5dc08dffd3b1ed6f",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-common/0.14.0/reef-common-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-runtime-local:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:0.14.0"
-                ],
                 "dependencies": [
                     "org.apache.avro:avro:1.7.4",
                     "org.ow2.asm:asm:5.0.4",
@@ -5564,20 +5565,20 @@
                     "org.codehaus.jackson:jackson-core-asl:1.8.8",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar"
                 ],
-                "sha256": "657e1acc384656c61ebb9255e7eb8d175742c45ba8464ce80f6887fb5aea7717"
+                "sha256": "657e1acc384656c61ebb9255e7eb8d175742c45ba8464ce80f6887fb5aea7717",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-runtime-local:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:jar:sources:0.14.0"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
@@ -5604,21 +5605,20 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:jar:sources:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar"
                 ],
-                "sha256": "8ff798dececcaac2c7bec67573f24986f7f246e849c512b6c830febc470fe00f"
+                "sha256": "8ff798dececcaac2c7bec67573f24986f7f246e849c512b6c830febc470fe00f",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-local/0.14.0/reef-runtime-local-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-runtime-yarn:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:0.14.0",
-                    "org.apache.reef:reef-utils-hadoop:0.14.0"
-                ],
                 "dependencies": [
                     "org.apache.avro:avro:1.7.4",
                     "org.ow2.asm:asm:5.0.4",
@@ -5646,21 +5646,21 @@
                     "org.codehaus.jackson:jackson-core-asl:1.8.8",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:0.14.0",
+                    "org.apache.reef:reef-utils-hadoop:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar"
                 ],
-                "sha256": "6211178601f8a934934d67bd19a6b2f011a6dd495f2524e1dc9ca3c46039e65e"
+                "sha256": "6211178601f8a934934d67bd19a6b2f011a6dd495f2524e1dc9ca3c46039e65e",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-runtime-yarn:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:jar:sources:0.14.0",
-                    "org.apache.reef:reef-utils-hadoop:jar:sources:0.14.0"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
@@ -5688,20 +5688,21 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:jar:sources:0.14.0",
+                    "org.apache.reef:reef-utils-hadoop:jar:sources:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar"
                 ],
-                "sha256": "1af847f292e89ff5330425f31d0a63e7be36d98b95b0fbb4ddec76ab099d8b16"
+                "sha256": "1af847f292e89ff5330425f31d0a63e7be36d98b95b0fbb4ddec76ab099d8b16",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-runtime-yarn/0.14.0/reef-runtime-yarn-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-utils-hadoop:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:0.14.0"
-                ],
                 "dependencies": [
                     "org.apache.avro:avro:1.7.4",
                     "org.ow2.asm:asm:5.0.4",
@@ -5728,20 +5729,20 @@
                     "org.codehaus.jackson:jackson-core-asl:1.8.8",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar"
                 ],
-                "sha256": "3790b740daec72e3b4b80680c3e4d9777db6d79db3e50da34a047e18e0e3f35a"
+                "sha256": "3790b740daec72e3b4b80680c3e4d9777db6d79db3e50da34a047e18e0e3f35a",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-utils-hadoop:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar",
-                "directDependencies": [
-                    "org.apache.reef:reef-common:jar:sources:0.14.0"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "com.google.code.findbugs:jsr305:jar:sources:3.0.2",
@@ -5768,62 +5769,58 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar",
+                "directDependencies": [
+                    "org.apache.reef:reef-common:jar:sources:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar"
                 ],
-                "sha256": "c7cd46b6fd45a454f5a340ac28dd5d277d0ffbb2864f328f4385c39781517e1b"
+                "sha256": "c7cd46b6fd45a454f5a340ac28dd5d277d0ffbb2864f328f4385c39781517e1b",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils-hadoop/0.14.0/reef-utils-hadoop-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:reef-utils:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar",
-                "directDependencies": [
-                    "javax.inject:javax.inject:1",
-                    "net.jcip:jcip-annotations:1.0"
-                ],
                 "dependencies": [
                     "net.jcip:jcip-annotations:1.0",
                     "javax.inject:javax.inject:1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar",
+                "directDependencies": [
+                    "javax.inject:javax.inject:1",
+                    "net.jcip:jcip-annotations:1.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar"
                 ],
-                "sha256": "2fa7f3b4241e816b33ba80e478f59f7fd1f4221764cc82dc030f9facbfd9cb7a"
+                "sha256": "2fa7f3b4241e816b33ba80e478f59f7fd1f4221764cc82dc030f9facbfd9cb7a",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:reef-utils:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar",
-                "directDependencies": [
-                    "javax.inject:javax.inject:jar:sources:1",
-                    "net.jcip:jcip-annotations:jar:sources:1.0"
-                ],
                 "dependencies": [
                     "net.jcip:jcip-annotations:jar:sources:1.0",
                     "javax.inject:javax.inject:jar:sources:1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar",
+                "directDependencies": [
+                    "javax.inject:javax.inject:jar:sources:1",
+                    "net.jcip:jcip-annotations:jar:sources:1.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar"
                 ],
-                "sha256": "5489283c31df1884546db9e4553735676da51bb60522408aadaf2037302a5d9d"
+                "sha256": "5489283c31df1884546db9e4553735676da51bb60522408aadaf2037302a5d9d",
+                "url": "https://jcenter.bintray.com/org/apache/reef/reef-utils/0.14.0/reef-utils-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:tang:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar",
-                "directDependencies": [
-                    "org.apache.avro:avro:1.7.4",
-                    "commons-configuration:commons-configuration:1.10",
-                    "commons-cli:commons-cli:1.3.1",
-                    "com.google.protobuf:protobuf-java:3.4.0",
-                    "javax.inject:javax.inject:1"
-                ],
                 "dependencies": [
                     "org.apache.avro:avro:1.7.4",
                     "commons-logging:commons-logging:1.2",
@@ -5839,24 +5836,24 @@
                     "javax.inject:javax.inject:1",
                     "org.codehaus.jackson:jackson-core-asl:1.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar",
+                "directDependencies": [
+                    "org.apache.avro:avro:1.7.4",
+                    "commons-configuration:commons-configuration:1.10",
+                    "commons-cli:commons-cli:1.3.1",
+                    "com.google.protobuf:protobuf-java:3.4.0",
+                    "javax.inject:javax.inject:1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/tang/0.14.0/tang-0.14.0.jar"
                 ],
-                "sha256": "958082feff6dbd0fb8a681f83baee16b146f1d861bc84999b3fa7f689ef9d553"
+                "sha256": "958082feff6dbd0fb8a681f83baee16b146f1d861bc84999b3fa7f689ef9d553",
+                "url": "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:tang:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar",
-                "directDependencies": [
-                    "commons-configuration:commons-configuration:jar:sources:1.10",
-                    "javax.inject:javax.inject:jar:sources:1",
-                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
-                    "org.apache.avro:avro:jar:sources:1.7.4",
-                    "commons-cli:commons-cli:jar:sources:1.3.1"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8",
@@ -5872,24 +5869,24 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar",
+                "directDependencies": [
+                    "commons-configuration:commons-configuration:jar:sources:1.10",
+                    "javax.inject:javax.inject:jar:sources:1",
+                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
+                    "org.apache.avro:avro:jar:sources:1.7.4",
+                    "commons-cli:commons-cli:jar:sources:1.3.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar"
                 ],
-                "sha256": "8f78cd4b706d796baa5502a71c3b9fc889a72926839c76dd4286d01510c5f353"
+                "sha256": "8f78cd4b706d796baa5502a71c3b9fc889a72926839c76dd4286d01510c5f353",
+                "url": "https://jcenter.bintray.com/org/apache/reef/tang/0.14.0/tang-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.reef:wake:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar",
-                "directDependencies": [
-                    "cglib:cglib:3.1",
-                    "org.apache.reef:tang:0.14.0",
-                    "net.jcip:jcip-annotations:1.0",
-                    "com.google.protobuf:protobuf-java:3.4.0",
-                    "io.netty:netty-all:4.1.22.Final"
-                ],
                 "dependencies": [
                     "org.apache.avro:avro:1.7.4",
                     "org.ow2.asm:asm:5.0.4",
@@ -5910,24 +5907,24 @@
                     "org.codehaus.jackson:jackson-core-asl:1.8.8",
                     "io.netty:netty-all:4.1.22.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar",
+                "directDependencies": [
+                    "cglib:cglib:3.1",
+                    "org.apache.reef:tang:0.14.0",
+                    "net.jcip:jcip-annotations:1.0",
+                    "com.google.protobuf:protobuf-java:3.4.0",
+                    "io.netty:netty-all:4.1.22.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar",
                     "https://maven.google.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/wake/0.14.0/wake-0.14.0.jar"
                 ],
-                "sha256": "7f4902d2ac788434f0f6e38f8fbe5a31b4b547a1e2fa342375efc18cd0c94bba"
+                "sha256": "7f4902d2ac788434f0f6e38f8fbe5a31b4b547a1e2fa342375efc18cd0c94bba",
+                "url": "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0.jar"
             },
             {
                 "coord": "org.apache.reef:wake:jar:sources:0.14.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar",
-                "directDependencies": [
-                    "net.jcip:jcip-annotations:jar:sources:1.0",
-                    "io.netty:netty-all:jar:sources:4.1.22.Final",
-                    "cglib:cglib:jar:sources:3.1",
-                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
-                    "org.apache.reef:tang:jar:sources:0.14.0"
-                ],
                 "dependencies": [
                     "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8",
@@ -5948,124 +5945,122 @@
                     "org.apache.avro:avro:jar:sources:1.7.4",
                     "commons-cli:commons-cli:jar:sources:1.3.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar",
+                "directDependencies": [
+                    "net.jcip:jcip-annotations:jar:sources:1.0",
+                    "io.netty:netty-all:jar:sources:4.1.22.Final",
+                    "cglib:cglib:jar:sources:3.1",
+                    "com.google.protobuf:protobuf-java:jar:sources:3.4.0",
+                    "org.apache.reef:tang:jar:sources:0.14.0"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar",
                     "https://maven.google.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar"
                 ],
-                "sha256": "b571f95f42cb87e089803efcb158c72d0b21812c4a701e4652d80dfb72f84b29"
+                "sha256": "b571f95f42cb87e089803efcb158c72d0b21812c4a701e4652d80dfb72f84b29",
+                "url": "https://jcenter.bintray.com/org/apache/reef/wake/0.14.0/wake-0.14.0-sources.jar"
             },
             {
                 "coord": "org.apache.xbean:xbean-reflect:3.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "log4j:log4j:1.2.17"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "log4j:log4j:1.2.17"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar",
                     "https://maven.google.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar",
                     "https://repo1.maven.org/maven2/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar"
                 ],
-                "sha256": "17e0efa187127034623197fb88c50c30d3baa62baa0f07d6ec693047ac92ec3b"
+                "sha256": "17e0efa187127034623197fb88c50c30d3baa62baa0f07d6ec693047ac92ec3b",
+                "url": "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4.jar"
             },
             {
                 "coord": "org.apache.xbean:xbean-reflect:jar:sources:3.4",
-                "file": "v1/https/jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "commons-logging:commons-logging-api:jar:sources:1.1",
                     "log4j:log4j:jar:sources:1.2.17"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "commons-logging:commons-logging-api:jar:sources:1.1",
                     "log4j:log4j:jar:sources:1.2.17"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar",
                     "https://maven.google.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar"
                 ],
-                "sha256": "1e6f89e8c5fc05b15a2def9b1414cac7e8c01e0b3dc25feece2bef4b67ef4de1"
+                "sha256": "1e6f89e8c5fc05b15a2def9b1414cac7e8c01e0b3dc25feece2bef4b67ef4de1",
+                "url": "https://jcenter.bintray.com/org/apache/xbean/xbean-reflect/3.4/xbean-reflect-3.4-sources.jar"
             },
             {
                 "coord": "org.apache.yetus:audience-annotations:0.5.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar",
                     "https://maven.google.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar",
                     "https://repo1.maven.org/maven2/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar"
                 ],
-                "sha256": "c82631f06c75d46bf6524d95f0d6c2e3aef1b3eb4a7b584ca296624ef0d474be"
+                "sha256": "c82631f06c75d46bf6524d95f0d6c2e3aef1b3eb4a7b584ca296624ef0d474be",
+                "url": "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0.jar"
             },
             {
                 "coord": "org.apache.yetus:audience-annotations:jar:sources:0.5.0",
-                "file": "v1/https/jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar",
                     "https://maven.google.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar"
                 ],
-                "sha256": "05d975529ac0d51ffd6c38376d59c29f79fca2f5c6bef76e096223c52d26928b"
+                "sha256": "05d975529ac0d51ffd6c38376d59c29f79fca2f5c6bef76e096223c52d26928b",
+                "url": "https://jcenter.bintray.com/org/apache/yetus/audience-annotations/0.5.0/audience-annotations-0.5.0-sources.jar"
             },
             {
                 "coord": "org.apache.zookeeper:zookeeper-jute:3.5.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.yetus:audience-annotations:0.5.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.yetus:audience-annotations:0.5.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar",
                     "https://maven.google.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar"
                 ],
-                "sha256": "7f0c9de0ee4747d0e34c3d2c8bc0da80a412c2c359520bc3754bd052afd38d69"
+                "sha256": "7f0c9de0ee4747d0e34c3d2c8bc0da80a412c2c359520bc3754bd052afd38d69",
+                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8.jar"
             },
             {
                 "coord": "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.yetus:audience-annotations:jar:sources:0.5.0"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.yetus:audience-annotations:jar:sources:0.5.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar",
                     "https://maven.google.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar"
                 ],
-                "sha256": "e2eaaac45d49af59aa68b9ed3dc3eb28a344eb1579749b4b7a029610264b13f9"
+                "sha256": "e2eaaac45d49af59aa68b9ed3dc3eb28a344eb1579749b4b7a029610264b13f9",
+                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper-jute/3.5.8/zookeeper-jute-3.5.8-sources.jar"
             },
             {
                 "coord": "org.apache.zookeeper:zookeeper:3.5.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar",
-                "directDependencies": [
-                    "org.slf4j:slf4j-log4j12:1.7.25",
-                    "org.slf4j:slf4j-api:1.7.30",
-                    "log4j:log4j:1.2.17",
-                    "io.netty:netty-transport-native-epoll:4.1.48.Final",
-                    "org.apache.zookeeper:zookeeper-jute:3.5.8",
-                    "org.apache.yetus:audience-annotations:0.5.0",
-                    "io.netty:netty-handler:4.1.48.Final"
-                ],
                 "dependencies": [
                     "io.netty:netty-transport:4.1.48.Final",
                     "org.slf4j:slf4j-log4j12:1.7.25",
@@ -6081,102 +6076,111 @@
                     "io.netty:netty-codec:4.1.48.Final",
                     "io.netty:netty-transport-native-unix-common:4.1.48.Final"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar",
+                "directDependencies": [
+                    "org.slf4j:slf4j-log4j12:1.7.25",
+                    "org.slf4j:slf4j-api:1.7.30",
+                    "log4j:log4j:1.2.17",
+                    "io.netty:netty-transport-native-epoll:4.1.48.Final",
+                    "org.apache.zookeeper:zookeeper-jute:3.5.8",
+                    "org.apache.yetus:audience-annotations:0.5.0",
+                    "io.netty:netty-handler:4.1.48.Final"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar",
                     "https://maven.google.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar",
                     "https://repo1.maven.org/maven2/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar"
                 ],
-                "sha256": "a606ccab2c476d638a25a386661cc7cf1943904461f818a20d94411fea93b852"
+                "sha256": "a606ccab2c476d638a25a386661cc7cf1943904461f818a20d94411fea93b852",
+                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8.jar"
             },
             {
                 "coord": "org.apache.zookeeper:zookeeper:jar:sources:3.5.8",
-                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
+                    "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "org.apache.yetus:audience-annotations:jar:sources:0.5.0",
+                    "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-handler:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport-native-epoll:jar:sources:4.1.48.Final",
                     "org.slf4j:slf4j-log4j12:jar:sources:1.7.25",
+                    "io.netty:netty-codec:jar:sources:4.1.48.Final",
                     "log4j:log4j:jar:sources:1.2.17",
+                    "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final",
+                    "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8"
                 ],
-                "dependencies": [
-                    "io.netty:netty-buffer:jar:sources:4.1.48.Final",
+                "directDependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
-                    "io.netty:netty-common:jar:sources:4.1.48.Final",
                     "org.apache.yetus:audience-annotations:jar:sources:0.5.0",
-                    "io.netty:netty-resolver:jar:sources:4.1.48.Final",
                     "io.netty:netty-handler:jar:sources:4.1.48.Final",
                     "io.netty:netty-transport-native-epoll:jar:sources:4.1.48.Final",
                     "org.slf4j:slf4j-log4j12:jar:sources:1.7.25",
-                    "io.netty:netty-codec:jar:sources:4.1.48.Final",
                     "log4j:log4j:jar:sources:1.2.17",
-                    "io.netty:netty-transport-native-unix-common:jar:sources:4.1.48.Final",
-                    "io.netty:netty-transport:jar:sources:4.1.48.Final",
                     "org.apache.zookeeper:zookeeper-jute:jar:sources:3.5.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar",
                     "https://maven.google.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar"
                 ],
-                "sha256": "64058b3e30c534b99ceb165cb38aa8321a8060a3b0ea941dc6e65ee1c7a4772c"
+                "sha256": "64058b3e30c534b99ceb165cb38aa8321a8060a3b0ea941dc6e65ee1c7a4772c",
+                "url": "https://jcenter.bintray.com/org/apache/zookeeper/zookeeper/3.5.8/zookeeper-3.5.8-sources.jar"
             },
             {
                 "coord": "org.asynchttpclient:async-http-client-netty-utils:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar",
                     "https://maven.google.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar"
                 ],
-                "sha256": "d62465c3d7f679498d5c2407a27f82bf51444d97c136db458551999eff3bed99"
+                "sha256": "d62465c3d7f679498d5c2407a27f82bf51444d97c136db458551999eff3bed99",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31.jar"
             },
             {
                 "coord": "org.asynchttpclient:async-http-client-netty-utils:jar:sources:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar",
                     "https://maven.google.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar"
                 ],
-                "sha256": "c02567af00a0de376751af0cec2c28417e066508a866e854cf4a9fa01af5df07"
+                "sha256": "c02567af00a0de376751af0cec2c28417e066508a866e854cf4a9fa01af5df07",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client-netty-utils/2.0.31/async-http-client-netty-utils-2.0.31-sources.jar"
             },
             {
                 "coord": "org.asynchttpclient:async-http-client:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "org.asynchttpclient:netty-codec-dns:2.0.31",
+                    "org.asynchttpclient:netty-resolver:2.0.31",
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.reactivestreams:reactive-streams:1.0.0",
                     "com.typesafe.netty:netty-reactive-streams:1.0.8",
                     "org.asynchttpclient:netty-resolver-dns:2.0.31",
                     "org.asynchttpclient:async-http-client-netty-utils:2.0.31"
                 ],
-                "dependencies": [
-                    "org.asynchttpclient:netty-codec-dns:2.0.31",
-                    "org.asynchttpclient:netty-resolver:2.0.31",
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.reactivestreams:reactive-streams:1.0.0",
                     "com.typesafe.netty:netty-reactive-streams:1.0.8",
@@ -6186,29 +6190,29 @@
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar",
                     "https://maven.google.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar"
                 ],
-                "sha256": "e069433cc58fbac4556f5a82ba86f10fd92962e5df32e63b7a707399f336eb6e"
+                "sha256": "e069433cc58fbac4556f5a82ba86f10fd92962e5df32e63b7a707399f336eb6e",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31.jar"
             },
             {
                 "coord": "org.asynchttpclient:async-http-client:jar:sources:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.asynchttpclient:netty-resolver-dns:jar:sources:2.0.31",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
+                    "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
+                    "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
                     "org.asynchttpclient:async-http-client-netty-utils:jar:sources:2.0.31",
                     "org.reactivestreams:reactive-streams:jar:sources:1.0.0",
                     "com.typesafe.netty:netty-reactive-streams:jar:sources:1.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.asynchttpclient:netty-resolver-dns:jar:sources:2.0.31",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30",
-                    "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
-                    "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
                     "org.asynchttpclient:async-http-client-netty-utils:jar:sources:2.0.31",
                     "org.reactivestreams:reactive-streams:jar:sources:1.0.0",
                     "com.typesafe.netty:netty-reactive-streams:jar:sources:1.0.8"
@@ -6216,297 +6220,325 @@
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar",
                     "https://maven.google.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar"
                 ],
-                "sha256": "601fd8a6eec8929e9a708f930105e080f77ff8615b7f1369972557eae417636d"
+                "sha256": "601fd8a6eec8929e9a708f930105e080f77ff8615b7f1369972557eae417636d",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/async-http-client/2.0.31/async-http-client-2.0.31-sources.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-codec-dns:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar"
                 ],
-                "sha256": "d0e6033a8489ae9db2293077c87ddaf3f99d80c73d0ecbf74ab29d08083a96b4"
+                "sha256": "d0e6033a8489ae9db2293077c87ddaf3f99d80c73d0ecbf74ab29d08083a96b4",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar"
                 ],
-                "sha256": "157b37aeaa3f615d3897c476701d40fe1417f2a0840675b1eb8d97b41ca7d0c7"
+                "sha256": "157b37aeaa3f615d3897c476701d40fe1417f2a0840675b1eb8d97b41ca7d0c7",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-codec-dns/2.0.31/netty-codec-dns-2.0.31-sources.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-resolver-dns:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar",
-                "directDependencies": [
-                    "org.asynchttpclient:netty-codec-dns:2.0.31",
-                    "org.asynchttpclient:netty-resolver:2.0.31",
-                    "org.slf4j:slf4j-api:1.7.30"
-                ],
                 "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30",
                     "org.asynchttpclient:netty-resolver:2.0.31",
                     "org.asynchttpclient:netty-codec-dns:2.0.31"
                 ],
+                "directDependencies": [
+                    "org.asynchttpclient:netty-codec-dns:2.0.31",
+                    "org.asynchttpclient:netty-resolver:2.0.31",
+                    "org.slf4j:slf4j-api:1.7.30"
+                ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar"
                 ],
-                "sha256": "8c76c8bc84887d2c38672c073272693375dbf800545daa271b56f44030ced906"
+                "sha256": "8c76c8bc84887d2c38672c073272693375dbf800545daa271b56f44030ced906",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-resolver-dns:jar:sources:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar",
-                "directDependencies": [
-                    "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
+                "dependencies": [
                     "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
+                    "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
-                "dependencies": [
-                    "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
+                "directDependencies": [
                     "org.asynchttpclient:netty-codec-dns:jar:sources:2.0.31",
+                    "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar"
                 ],
-                "sha256": "68924cc67f13abeda0a564b328e707d51840a201460f26d75530e71118226446"
+                "sha256": "68924cc67f13abeda0a564b328e707d51840a201460f26d75530e71118226446",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver-dns/2.0.31/netty-resolver-dns-2.0.31-sources.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-resolver:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar"
                 ],
-                "sha256": "2c6b7b984501b46a93429240f596b00523e819565a7f846d640e25951e831189"
+                "sha256": "2c6b7b984501b46a93429240f596b00523e819565a7f846d640e25951e831189",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31.jar"
             },
             {
                 "coord": "org.asynchttpclient:netty-resolver:jar:sources:2.0.31",
-                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.slf4j:slf4j-api:jar:sources:1.7.30"
                 ],
                 "exclusions": [
                     "io.netty:*"
                 ],
-                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar",
                     "https://maven.google.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar",
                     "https://repo1.maven.org/maven2/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar"
                 ],
-                "sha256": "22f8b76f5501a676ccd67e97542004db3a044bcde7866bdb6ef16543e427b001"
+                "sha256": "22f8b76f5501a676ccd67e97542004db3a044bcde7866bdb6ef16543e427b001",
+                "url": "https://jcenter.bintray.com/org/asynchttpclient/netty-resolver/2.0.31/netty-resolver-2.0.31-sources.jar"
             },
             {
-                "coord": "org.bouncycastle:bcpkix-jdk15on:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar",
+                "coord": "org.bitbucket.b_c:jose4j:0.7.0",
+                "dependencies": [
+                    "org.slf4j:slf4j-api:1.7.30"
+                ],
                 "directDependencies": [
-                    "org.bouncycastle:bcprov-jdk15on:jar:1.61"
+                    "org.slf4j:slf4j-api:1.7.30"
                 ],
+                "file": "v1/https/jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0.jar",
+                "mirror_urls": [
+                    "https://jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0.jar",
+                    "https://maven.google.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0.jar",
+                    "https://repo1.maven.org/maven2/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0.jar"
+                ],
+                "sha256": "eb14f69c0395d4a106c6c46fe6dff080c4608ccabc99b1f03933d374383d9bbe",
+                "url": "https://jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0.jar"
+            },
+            {
+                "coord": "org.bitbucket.b_c:jose4j:jar:sources:0.7.0",
+                "dependencies": [
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
+                ],
+                "directDependencies": [
+                    "org.slf4j:slf4j-api:jar:sources:1.7.30"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0-sources.jar",
+                "mirror_urls": [
+                    "https://jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0-sources.jar",
+                    "https://maven.google.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0-sources.jar",
+                    "https://repo1.maven.org/maven2/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0-sources.jar"
+                ],
+                "sha256": "8d04c27bbda1dc9eeb860a389fe3ddb50d29b0ae85a04526b97792220c2ab283",
+                "url": "https://jcenter.bintray.com/org/bitbucket/b_c/jose4j/0.7.0/jose4j-0.7.0-sources.jar"
+            },
+            {
+                "coord": "org.bouncycastle:bcpkix-jdk15on:1.61",
                 "dependencies": [
                     "org.bouncycastle:bcprov-jdk15on:jar:1.61"
                 ],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar",
+                "directDependencies": [
+                    "org.bouncycastle:bcprov-jdk15on:jar:1.61"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar",
                     "https://maven.google.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar"
                 ],
-                "sha256": "326eb81c2a0cb0d665733a9cc7c03988081101ad17d1453b334368453658591f"
+                "sha256": "326eb81c2a0cb0d665733a9cc7c03988081101ad17d1453b334368453658591f",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61.jar"
             },
             {
                 "coord": "org.bouncycastle:bcpkix-jdk15on:jar:sources:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.bouncycastle:bcprov-jdk15on:jar:sources:1.61"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.bouncycastle:bcprov-jdk15on:jar:sources:1.61"
                 ],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar",
                     "https://maven.google.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar"
                 ],
-                "sha256": "5006164e85d3416c6ce600a07ed2c7b11648d119b57c2356e7eaad4fd382f284"
+                "sha256": "5006164e85d3416c6ce600a07ed2c7b11648d119b57c2356e7eaad4fd382f284",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcpkix-jdk15on/1.61/bcpkix-jdk15on-1.61-sources.jar"
             },
             {
                 "coord": "org.bouncycastle:bcprov-ext-jdk15on:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar",
                     "https://maven.google.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar"
                 ],
-                "sha256": "d363a343683d0ffe70ce1ebb9597ccb50ea4e1da5f89f06802b3a40966904d34"
+                "sha256": "d363a343683d0ffe70ce1ebb9597ccb50ea4e1da5f89f06802b3a40966904d34",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61.jar"
             },
             {
                 "coord": "org.bouncycastle:bcprov-ext-jdk15on:jar:sources:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar",
                     "https://maven.google.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar"
                 ],
-                "sha256": "3a8efda3c9bfd36f03c26ad2dc883b7ee8e423a49fbbbaaa56a169bc7f2dbd8b"
+                "sha256": "3a8efda3c9bfd36f03c26ad2dc883b7ee8e423a49fbbbaaa56a169bc7f2dbd8b",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-ext-jdk15on/1.61/bcprov-ext-jdk15on-1.61-sources.jar"
             },
             {
                 "coord": "org.bouncycastle:bcprov-jdk15on:jar:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar",
                     "https://maven.google.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar"
                 ],
-                "sha256": "dba6e408f205215ad1a89b70b37353d3cdae4ec61037e1feee885704e2413458"
+                "sha256": "dba6e408f205215ad1a89b70b37353d3cdae4ec61037e1feee885704e2413458",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61.jar"
             },
             {
                 "coord": "org.bouncycastle:bcprov-jdk15on:jar:sources:1.61",
-                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar",
                     "https://maven.google.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar",
                     "https://repo1.maven.org/maven2/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar"
                 ],
-                "sha256": "3a8efda3c9bfd36f03c26ad2dc883b7ee8e423a49fbbbaaa56a169bc7f2dbd8b"
+                "sha256": "3a8efda3c9bfd36f03c26ad2dc883b7ee8e423a49fbbbaaa56a169bc7f2dbd8b",
+                "url": "https://jcenter.bintray.com/org/bouncycastle/bcprov-jdk15on/1.61/bcprov-jdk15on-1.61-sources.jar"
             },
             {
                 "coord": "org.codehaus.jackson:jackson-core-asl:1.8.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar",
                     "https://maven.google.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar"
                 ],
-                "sha256": "96b394f135bf396679681aca6716d8bea14a97cf306d3738a053c43d07a1308b"
+                "sha256": "96b394f135bf396679681aca6716d8bea14a97cf306d3738a053c43d07a1308b",
+                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8.jar"
             },
             {
                 "coord": "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar",
                     "https://maven.google.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar"
                 ],
-                "sha256": "2c30bf0e0c4ee8bdadfe69c8be23cbd3cbe3982c7745117f6ac4bb1343ac3cca"
+                "sha256": "2c30bf0e0c4ee8bdadfe69c8be23cbd3cbe3982c7745117f6ac4bb1343ac3cca",
+                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-core-asl/1.8.8/jackson-core-asl-1.8.8-sources.jar"
             },
             {
                 "coord": "org.codehaus.jackson:jackson-mapper-asl:1.8.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.jackson:jackson-core-asl:1.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.jackson:jackson-core-asl:1.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar",
                     "https://maven.google.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar"
                 ],
-                "sha256": "56436abd3e06c45e496b8604fd3f3b0f22451a9b5de8433b6f8b416e7a14a048"
+                "sha256": "56436abd3e06c45e496b8604fd3f3b0f22451a9b5de8433b6f8b416e7a14a048",
+                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8.jar"
             },
             {
                 "coord": "org.codehaus.jackson:jackson-mapper-asl:jar:sources:1.8.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.jackson:jackson-core-asl:jar:sources:1.8.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar",
                     "https://maven.google.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar"
                 ],
-                "sha256": "15817a2897fe38a58f7bfa4effeee5a8215358902c210c8a4afa0889c0b7ae2d"
+                "sha256": "15817a2897fe38a58f7bfa4effeee5a8215358902c210c8a4afa0889c0b7ae2d",
+                "url": "https://jcenter.bintray.com/org/codehaus/jackson/jackson-mapper-asl/1.8.8/jackson-mapper-asl-1.8.8-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-archiver:2.4.4",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar",
-                "directDependencies": [
-                    "org.apache.commons:commons-compress:1.14",
-                    "org.codehaus.plexus:plexus-container-default:1.5.5",
-                    "org.codehaus.plexus:plexus-io:2.0.10",
-                    "org.codehaus.plexus:plexus-utils:3.0.8"
-                ],
                 "dependencies": [
                     "commons-logging:commons-logging-api:1.1",
                     "org.apache.commons:commons-compress:1.14",
@@ -6520,23 +6552,23 @@
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar",
+                "directDependencies": [
+                    "org.apache.commons:commons-compress:1.14",
+                    "org.codehaus.plexus:plexus-container-default:1.5.5",
+                    "org.codehaus.plexus:plexus-io:2.0.10",
+                    "org.codehaus.plexus:plexus-utils:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar"
                 ],
-                "sha256": "b1eecb2db50665f9dc0f0e17fb94f9663268186658bdc272d0b88f60273d4f50"
+                "sha256": "b1eecb2db50665f9dc0f0e17fb94f9663268186658bdc272d0b88f60273d4f50",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-archiver:jar:sources:2.4.4",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar",
-                "directDependencies": [
-                    "org.apache.commons:commons-compress:jar:sources:1.14",
-                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                    "org.codehaus.plexus:plexus-io:jar:sources:2.0.10",
-                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
-                ],
                 "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
                     "commons-logging:commons-logging-api:jar:sources:1.1",
@@ -6550,315 +6582,318 @@
                     "junit:junit:jar:sources:4.12",
                     "org.codehaus.plexus:plexus-io:jar:sources:2.0.10"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar",
+                "directDependencies": [
+                    "org.apache.commons:commons-compress:jar:sources:1.14",
+                    "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
+                    "org.codehaus.plexus:plexus-io:jar:sources:2.0.10",
+                    "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar"
                 ],
-                "sha256": "fc3901f8e097d5b5fae0b8e0812956d847e032513a662fe8ef7e206de6177381"
+                "sha256": "fc3901f8e097d5b5fae0b8e0812956d847e032513a662fe8ef7e206de6177381",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-archiver/2.4.4/plexus-archiver-2.4.4-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-classworlds:2.2.2",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar"
                 ],
-                "sha256": "13a90763640e445ffa432ce9586e416572645c3ed4db6a860fe0d28256ad40ce"
+                "sha256": "13a90763640e445ffa432ce9586e416572645c3ed4db6a860fe0d28256ad40ce",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar"
                 ],
-                "sha256": "178bab3d2b950fd868854e207730429c7f1228932dbf86589313bc05bd820d6e"
+                "sha256": "178bab3d2b950fd868854e207730429c7f1228932dbf86589313bc05bd820d6e",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-classworlds/2.2.2/plexus-classworlds-2.2.2-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-container-default:1.5.5",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "commons-logging:commons-logging-api:1.1",
                     "junit:junit:4.12",
+                    "log4j:log4j:1.2.17",
+                    "org.hamcrest:hamcrest-core:1.3",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "com.google.collections:google-collections:1.0",
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "dependencies": [
-                    "commons-logging:commons-logging-api:1.1",
+                "directDependencies": [
                     "junit:junit:4.12",
-                    "log4j:log4j:1.2.17",
-                    "org.hamcrest:hamcrest-core:1.3",
                     "org.codehaus.plexus:plexus-utils:3.0.8",
                     "com.google.collections:google-collections:1.0",
                     "org.apache.xbean:xbean-reflect:3.4",
                     "org.codehaus.plexus:plexus-classworlds:2.2.2"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar"
                 ],
-                "sha256": "69197486cd80beb54b4e0fcabaa325ec2d4e2636e9b245c472435c87a10931cf"
+                "sha256": "69197486cd80beb54b4e0fcabaa325ec2d4e2636e9b245c472435c87a10931cf",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-container-default:jar:sources:1.5.5",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
+                    "commons-logging:commons-logging-api:jar:sources:1.1",
                     "com.google.collections:google-collections:jar:sources:1.0",
+                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
+                    "log4j:log4j:jar:sources:1.2.17",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.apache.xbean:xbean-reflect:jar:sources:3.4",
-                    "commons-logging:commons-logging-api:jar:sources:1.1",
                     "com.google.collections:google-collections:jar:sources:1.0",
-                    "org.hamcrest:hamcrest-core:jar:sources:1.3",
-                    "log4j:log4j:jar:sources:1.2.17",
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
                     "org.codehaus.plexus:plexus-classworlds:jar:sources:2.2.2",
                     "junit:junit:jar:sources:4.12"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar"
                 ],
-                "sha256": "8cb58c40e61a3b40aa22802e7d00007f08dd4f58111f0f46e7515d3736c0cb54"
+                "sha256": "8cb58c40e61a3b40aa22802e7d00007f08dd4f58111f0f46e7515d3736c0cb54",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-container-default/1.5.5/plexus-container-default-1.5.5-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-interactivity-api:1.0-alpha-4",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar",
-                "directDependencies": [
+                "dependencies": [
                     "classworlds:classworlds:1.1-alpha-2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "classworlds:classworlds:1.1-alpha-2"
                 ],
                 "exclusions": [
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar"
                 ],
-                "sha256": "4f60eb379f93d8b616bc3b4d299f466bc54fcced959f7ad082dae78b89d6a3f0"
+                "sha256": "4f60eb379f93d8b616bc3b4d299f466bc54fcced959f7ad082dae78b89d6a3f0",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-interactivity-api:jar:sources:1.0-alpha-4",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "classworlds:classworlds:jar:sources:1.1-alpha-2"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "classworlds:classworlds:jar:sources:1.1-alpha-2"
                 ],
                 "exclusions": [
                     "plexus:plexus-utils",
                     "org.codehaus.plexus:plexus-container-default"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar"
                 ],
-                "sha256": "2eae2dc145b8dca70671f4607255b5419b6609741c753b925debcef598d56206"
+                "sha256": "2eae2dc145b8dca70671f4607255b5419b6609741c753b925debcef598d56206",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-interactivity-api/1.0-alpha-4/plexus-interactivity-api-1.0-alpha-4-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-io:2.0.10",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar"
                 ],
-                "sha256": "195b9c6631473148f32c40b3e71da0409f6ec415b6ba1fff8593a8399f5aaf05"
+                "sha256": "195b9c6631473148f32c40b3e71da0409f6ec415b6ba1fff8593a8399f5aaf05",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-io:jar:sources:2.0.10",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8"
                 ],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar"
                 ],
-                "sha256": "9a0f04141f81f632a8dde3ff47cb2653568d30efc59207e37a2d8716c2704410"
+                "sha256": "9a0f04141f81f632a8dde3ff47cb2653568d30efc59207e37a2d8716c2704410",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-io/2.0.10/plexus-io-2.0.10-sources.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-utils:3.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar"
                 ],
-                "sha256": "6c040032841fe6b23612c7a4b52347a4a115fdde748086c399a154b4b108e56b"
+                "sha256": "6c040032841fe6b23612c7a4b52347a4a115fdde748086c399a154b4b108e56b",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8.jar"
             },
             {
                 "coord": "org.codehaus.plexus:plexus-utils:jar:sources:3.0.8",
-                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar",
                     "https://maven.google.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar",
                     "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar"
                 ],
-                "sha256": "771c29a752b844faf0979082883abae4d9c63292904cb38431d4f15587a623de"
+                "sha256": "771c29a752b844faf0979082883abae4d9c63292904cb38431d4f15587a623de",
+                "url": "https://jcenter.bintray.com/org/codehaus/plexus/plexus-utils/3.0.8/plexus-utils-3.0.8-sources.jar"
             },
             {
                 "coord": "org.eclipse.jdt:core:3.1.1",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar",
                     "https://maven.google.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar"
                 ],
-                "sha256": "f9e39cf7326b60d1d3016ed90fadc049f71d312c97aa97cbaaf851d6376730ba"
+                "sha256": "f9e39cf7326b60d1d3016ed90fadc049f71d312c97aa97cbaaf851d6376730ba",
+                "url": "https://jcenter.bintray.com/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-continuation:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar"
                 ],
-                "sha256": "57ded3ff608fecda070407b67dd1602610018e4e5362d8687a420e41bbcc0db6"
+                "sha256": "57ded3ff608fecda070407b67dd1602610018e4e5362d8687a420e41bbcc0db6",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-continuation:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "1642a6b00132a5687dcfc8bb59c707d39a9ddef020e5a877a22ad24ab234a686"
+                "sha256": "1642a6b00132a5687dcfc8bb59c707d39a9ddef020e5a877a22ad24ab234a686",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-continuation/9.4.6.v20170531/jetty-continuation-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-http:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar"
                 ],
-                "sha256": "2e661446bc74a55dc58c4f47314abdaaae20669fcbc858daee513473dbbcd97d"
+                "sha256": "2e661446bc74a55dc58c4f47314abdaaae20669fcbc858daee513473dbbcd97d",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "02ae8c61296f8b7fc1ddac50ee8f24d827274648ce3f0f2a2d62981a5ec00b4d"
+                "sha256": "02ae8c61296f8b7fc1ddac50ee8f24d827274648ce3f0f2a2d62981a5ec00b4d",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-http/9.4.6.v20170531/jetty-http-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar",
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar"
                 ],
-                "sha256": "6357631d9da902ac82f9bec39137e301fd6aeff65f43c4d0ef918e3d2845e440"
+                "sha256": "6357631d9da902ac82f9bec39137e301fd6aeff65f43c4d0ef918e3d2845e440",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "2fb3bd653a4b9b46b7ba2cb3e5a185467395f3735f85c7202b3ca9b8e0250410"
+                "sha256": "2fb3bd653a4b9b46b7ba2cb3e5a185467395f3735f85c7202b3ca9b8e0250410",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-io/9.4.6.v20170531/jetty-io-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-security:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-server:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-server:9.4.6.v20170531",
@@ -6866,20 +6901,20 @@
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:3.1.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-server:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar"
                 ],
-                "sha256": "d4f58c78bef67eb32086061b29b79808814f13e49790691c32e3fde2c7ad37ca"
+                "sha256": "d4f58c78bef67eb32086061b29b79808814f13e49790691c32e3fde2c7ad37ca",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-security:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-server:jar:sources:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:jar:sources:3.1.0",
@@ -6887,64 +6922,64 @@
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-server:jar:sources:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "b1e161ad9e43cb4c2bb60adfb4946fdfa7eaaa7f93ea0f47c4e895b3715c3e1e"
+                "sha256": "b1e161ad9e43cb4c2bb60adfb4946fdfa7eaaa7f93ea0f47c4e895b3715c3e1e",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-security/9.4.6.v20170531/jetty-security-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-server:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar",
-                "directDependencies": [
-                    "javax.servlet:javax.servlet-api:3.1.0",
-                    "org.eclipse.jetty:jetty-http:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:3.1.0",
                     "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-http:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar",
+                "directDependencies": [
+                    "javax.servlet:javax.servlet-api:3.1.0",
+                    "org.eclipse.jetty:jetty-http:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar"
                 ],
-                "sha256": "0f130457245b7561efd7fcc8e74dd49845ed716acde3f45f18eadf43afbcf593"
+                "sha256": "0f130457245b7561efd7fcc8e74dd49845ed716acde3f45f18eadf43afbcf593",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-server:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:jar:sources:3.1.0",
                     "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531"
                 ],
-                "dependencies": [
-                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
+                "directDependencies": [
                     "javax.servlet:javax.servlet-api:jar:sources:3.1.0",
                     "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "bb706974a23609c8d5feb226ba09919cad62412ffe51059df4c828b87ce33678"
+                "sha256": "bb706974a23609c8d5feb226ba09919cad62412ffe51059df4c828b87ce33678",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-server/9.4.6.v20170531/jetty-server-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-servlet:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-security:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-security:9.4.6.v20170531",
@@ -6953,20 +6988,20 @@
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:3.1.0"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-security:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar"
                 ],
-                "sha256": "2aa220dd22e5572af6e6922b259fdc35a4c7575d021bcc5c1d581b6d9096754d"
+                "sha256": "2aa220dd22e5572af6e6922b259fdc35a4c7575d021bcc5c1d581b6d9096754d",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-servlet:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-security:jar:sources:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
                     "javax.servlet:javax.servlet-api:jar:sources:3.1.0",
@@ -6975,331 +7010,328 @@
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-security:jar:sources:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "95593e64a57699bc0b9e892bf3615b45ff22331248122fa86d97c0f98fce2135"
+                "sha256": "95593e64a57699bc0b9e892bf3615b45ff22331248122fa86d97c0f98fce2135",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlet/9.4.6.v20170531/jetty-servlet-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-servlets:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-continuation:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-http:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-continuation:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-http:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-continuation:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-http:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-io:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-util:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar"
                 ],
-                "sha256": "fe74531244c452cc27dd2c56f472ce20e89b828cb9b3f3a954438625cfd0eb3c"
+                "sha256": "fe74531244c452cc27dd2c56f472ce20e89b828cb9b3f3a954438625cfd0eb3c",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-servlets:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar",
-                "directDependencies": [
-                    "org.eclipse.jetty:jetty-continuation:jar:sources:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
-                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
-                ],
                 "dependencies": [
                     "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-continuation:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
                     "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531"
                 ],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar",
+                "directDependencies": [
+                    "org.eclipse.jetty:jetty-continuation:jar:sources:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-http:jar:sources:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-io:jar:sources:9.4.6.v20170531",
+                    "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "60616baa2eec88acab365bb87bb77ea4d63f8f843a1f299362c89e88dc1a1462"
+                "sha256": "60616baa2eec88acab365bb87bb77ea4d63f8f843a1f299362c89e88dc1a1462",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-servlets/9.4.6.v20170531/jetty-servlets-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-util:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar"
                 ],
-                "sha256": "1616ce767bbd50b9dcad7c9ff074e8bc9736957d3bd4bd82362452b1461b2acc"
+                "sha256": "1616ce767bbd50b9dcad7c9ff074e8bc9736957d3bd4bd82362452b1461b2acc",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531.jar"
             },
             {
                 "coord": "org.eclipse.jetty:jetty-util:jar:sources:9.4.6.v20170531",
-                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar",
                     "https://maven.google.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar",
                     "https://repo1.maven.org/maven2/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar"
                 ],
-                "sha256": "b26027c76ad204360e8e53e9732e27e67ab4c575221005f9f4a4d05d0e22704d"
+                "sha256": "b26027c76ad204360e8e53e9732e27e67ab4c575221005f9f4a4d05d0e22704d",
+                "url": "https://jcenter.bintray.com/org/eclipse/jetty/jetty-util/9.4.6.v20170531/jetty-util-9.4.6.v20170531-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar",
                     "https://maven.google.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar"
                 ],
-                "sha256": "32a44ed0258c00bb8f0acf7e4dbf000a377bd48702465f6195f878a6dc2024d6"
+                "sha256": "32a44ed0258c00bb8f0acf7e4dbf000a377bd48702465f6195f878a6dc2024d6",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32.jar"
             },
             {
                 "coord": "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar"
                 ],
-                "sha256": "d70bd380f6510135e691976e1f2a2339231ea29e5db8e85629b5caa80f2acee4"
+                "sha256": "d70bd380f6510135e691976e1f2a2339231ea29e5db8e85629b5caa80f2acee4",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/aopalliance-repackaged/2.5.0-b32/aopalliance-repackaged-2.5.0-b32-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar",
                     "https://maven.google.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar"
                 ],
-                "sha256": "437c92cf50a0efa6b501b8939b5b92ede7cfe4455cf06b68ec69d1b21ab921ed"
+                "sha256": "437c92cf50a0efa6b501b8939b5b92ede7cfe4455cf06b68ec69d1b21ab921ed",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32.jar"
             },
             {
                 "coord": "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "javax.inject:javax.inject"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar"
                 ],
-                "sha256": "8d0950b6f017a935536c9190221357972fa9109bed408e432b3857354314e59e"
+                "sha256": "8d0950b6f017a935536c9190221357972fa9109bed408e432b3857354314e59e",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/external/javax.inject/2.5.0-b32/javax.inject-2.5.0-b32-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-api:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar",
-                "directDependencies": [
-                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
-                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32",
                     "org.glassfish.hk2:hk2-utils:2.5.0-b32"
                 ],
+                "directDependencies": [
+                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
+                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
+                ],
                 "exclusions": [
                     "javax.inject:javax.inject"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar"
                 ],
-                "sha256": "b3fe4f295ab8e74ea9d641717dc55e5768f1e5db3709e84235346a4d6bcde5c2"
+                "sha256": "b3fe4f295ab8e74ea9d641717dc55e5768f1e5db3709e84235346a4d6bcde5c2",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32"
                 ],
                 "exclusions": [
                     "javax.inject:javax.inject"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar"
                 ],
-                "sha256": "b1c6e233d1f3dc16636f8e3d8942caaa67d5b9dbfe550561e50a4bdfe8412d90"
+                "sha256": "b1c6e233d1f3dc16636f8e3d8942caaa67d5b9dbfe550561e50a4bdfe8412d90",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-api/2.5.0-b32/hk2-api-2.5.0-b32-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-locator:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
                     "org.javassist:javassist:3.20.0-GA",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
                     "org.glassfish.hk2:hk2-utils:2.5.0-b32",
+                    "javax.inject:javax.inject:1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
                     "org.javassist:javassist:3.20.0-GA",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
                     "org.glassfish.hk2:hk2-utils:2.5.0-b32",
-                    "javax.inject:javax.inject:1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar"
                 ],
-                "sha256": "27cacf80e8c088cc50f73b56344b779bdb7418e590a037659ab66b2b0cd9c492"
+                "sha256": "27cacf80e8c088cc50f73b56344b779bdb7418e590a037659ab66b2b0cd9c492",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "javax.inject:javax.inject:jar:sources:1",
                     "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "dependencies": [
-                    "javax.inject:javax.inject:jar:sources:1",
+                "directDependencies": [
                     "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar"
                 ],
-                "sha256": "66a4b4936ffa3b42cfa067f54688fe90d36d815b116cc4f47cee93dfe3f0a1e4"
+                "sha256": "66a4b4936ffa3b42cfa067f54688fe90d36d815b116cc4f47cee93dfe3f0a1e4",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-locator/2.5.0-b32/hk2-locator-2.5.0-b32-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-utils:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar",
-                "directDependencies": [],
                 "dependencies": [],
+                "directDependencies": [],
                 "exclusions": [
                     "javax.inject:javax.inject"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar"
                 ],
-                "sha256": "3912c470e621eb3e469c111f4c9a4dee486e2ce9db09a65b7609e006b6c3d38e"
+                "sha256": "3912c470e621eb3e469c111f4c9a4dee486e2ce9db09a65b7609e006b6c3d38e",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32.jar"
             },
             {
                 "coord": "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "javax.inject:javax.inject:jar:sources:1"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "javax.inject:javax.inject:jar:sources:1"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar"
                 ],
-                "sha256": "a22d3ca7e5c3b468679e733cd26a2be5dbe5058e9683477aa354e3043b1611db"
+                "sha256": "a22d3ca7e5c3b468679e733cd26a2be5dbe5058e9683477aa354e3043b1611db",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/hk2-utils/2.5.0-b32/hk2-utils-2.5.0-b32-sources.jar"
             },
             {
                 "coord": "org.glassfish.hk2:osgi-resource-locator:1.0.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar",
                     "https://maven.google.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar"
                 ],
-                "sha256": "775003be577e8806f51b6e442be1033d83be2cb2207227b349be0bf16e6c0843"
+                "sha256": "775003be577e8806f51b6e442be1033d83be2cb2207227b349be0bf16e6c0843",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1.jar"
             },
             {
                 "coord": "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar",
                     "https://maven.google.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar"
                 ],
-                "sha256": "359eb741777355f2e58067a079d55ab7099ba29ff32f7c27d3458ecaaf461077"
+                "sha256": "359eb741777355f2e58067a079d55ab7099ba29ff32f7c27d3458ecaaf461077",
+                "url": "https://jcenter.bintray.com/org/glassfish/hk2/osgi-resource-locator/1.0.1/osgi-resource-locator-1.0.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.bundles.repackaged:jersey-guava:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar"
                 ],
-                "sha256": "8a88a8ebae65cb4d77830b40f681bf742b55ec62e7a44cf91b8577a9396b9f81"
+                "sha256": "8a88a8ebae65cb4d77830b40f681bf742b55ec62e7a44cf91b8577a9396b9f81",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar",
-                "directDependencies": [],
                 "dependencies": [],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar",
+                "directDependencies": [],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar"
                 ],
-                "sha256": "3cb99dd2d809015a052a465824fec1c18c3d1e571a3e2e6447db0f243889578c"
+                "sha256": "3cb99dd2d809015a052a465824fec1c18c3d1e571a3e2e6447db0f243889578c",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/bundles/repackaged/jersey-guava/2.25.1/jersey-guava-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.containers:jersey-container-servlet-core:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar",
-                "directDependencies": [
-                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
-                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1",
-                    "org.glassfish.jersey.core:jersey-server:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
@@ -7318,26 +7350,26 @@
                     "org.glassfish.jersey.core:jersey-common:2.25.1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
+                "directDependencies": [
+                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
+                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1",
+                    "org.glassfish.jersey.core:jersey-server:2.25.1"
+                ],
                 "exclusions": [
                     "javax.servlet:servlet-api"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar"
                 ],
-                "sha256": "232f4f4e59e5944098351379a12aecc715906831c96a855624a81da552192ac4"
+                "sha256": "232f4f4e59e5944098351379a12aecc715906831c96a855624a81da552192ac4",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.containers:jersey-container-servlet-core:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar",
-                "directDependencies": [
-                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
-                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
-                    "org.glassfish.jersey.core:jersey-server:jar:sources:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.jersey.media:jersey-media-jaxb:jar:sources:2.25.1",
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
@@ -7356,26 +7388,26 @@
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
+                "directDependencies": [
+                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
+                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
+                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
+                    "org.glassfish.jersey.core:jersey-server:jar:sources:2.25.1"
+                ],
                 "exclusions": [
                     "javax.servlet:servlet-api"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar"
                 ],
-                "sha256": "306c03b544ca52f64af61d85e840e4919f898783569af3a01a9931544a942618"
+                "sha256": "306c03b544ca52f64af61d85e840e4919f898783569af3a01a9931544a942618",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet-core/2.25.1/jersey-container-servlet-core-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.containers:jersey-container-servlet:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar",
-                "directDependencies": [
-                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
-                    "org.glassfish.jersey.containers:jersey-container-servlet-core:2.25.1",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1",
-                    "org.glassfish.jersey.core:jersey-server:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
@@ -7395,23 +7427,23 @@
                     "org.glassfish.jersey.core:jersey-common:2.25.1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar",
+                "directDependencies": [
+                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
+                    "org.glassfish.jersey.containers:jersey-container-servlet-core:2.25.1",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1",
+                    "org.glassfish.jersey.core:jersey-server:2.25.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar"
                 ],
-                "sha256": "3669c50bef23aeeabdae02e5e4b214c9f1eb1019fa4d559f2eeadb563ba598e4"
+                "sha256": "3669c50bef23aeeabdae02e5e4b214c9f1eb1019fa4d559f2eeadb563ba598e4",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.containers:jersey-container-servlet:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar",
-                "directDependencies": [
-                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                    "org.glassfish.jersey.containers:jersey-container-servlet-core:jar:sources:2.25.1",
-                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
-                    "org.glassfish.jersey.core:jersey-server:jar:sources:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.jersey.media:jersey-media-jaxb:jar:sources:2.25.1",
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
@@ -7431,24 +7463,23 @@
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar",
+                "directDependencies": [
+                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
+                    "org.glassfish.jersey.containers:jersey-container-servlet-core:jar:sources:2.25.1",
+                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
+                    "org.glassfish.jersey.core:jersey-server:jar:sources:2.25.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar"
                 ],
-                "sha256": "8ab513341ead05cfa578e50aea98deb117ba98310763de1cc7753e9215d7eb18"
+                "sha256": "8ab513341ead05cfa578e50aea98deb117ba98310763de1cc7753e9215d7eb18",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/containers/jersey-container-servlet/2.25.1/jersey-container-servlet-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-client:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar",
-                "directDependencies": [
-                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-locator:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-api:2.5.0-b32",
-                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
@@ -7463,24 +7494,24 @@
                     "org.glassfish.jersey.core:jersey-common:2.25.1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar",
+                "directDependencies": [
+                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-locator:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-api:2.5.0-b32",
+                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar"
                 ],
-                "sha256": "10671e430dc7c841eb0bc54c9f3e265dbb60e9f85efaad71d1e39807057e405c"
+                "sha256": "10671e430dc7c841eb0bc54c9f3e265dbb60e9f85efaad71d1e39807057e405c",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-client:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar",
-                "directDependencies": [
-                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
-                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                    "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
-                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
-                    "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
                     "javax.inject:javax.inject:jar:sources:1",
@@ -7495,202 +7526,202 @@
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar",
+                "directDependencies": [
+                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
+                    "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
+                    "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
+                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
+                    "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar"
                 ],
-                "sha256": "e28da8ea2d025fbbc52c77cb45f9c1b09346e8d3850844f4d7a22d08e1eaba07"
+                "sha256": "e28da8ea2d025fbbc52c77cb45f9c1b09346e8d3850844f4d7a22d08e1eaba07",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-client/2.25.1/jersey-client-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-common:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
+                    "org.javassist:javassist:3.20.0-GA",
                     "org.glassfish.jersey.bundles.repackaged:jersey-guava:2.25.1",
                     "org.glassfish.hk2:hk2-locator:2.5.0-b32",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:1.3.2",
-                    "javax.ws.rs:javax.ws.rs-api:2.0.1"
+                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
+                    "javax.inject:javax.inject:1",
+                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                    "org.javassist:javassist:3.20.0-GA",
                     "org.glassfish.jersey.bundles.repackaged:jersey-guava:2.25.1",
                     "org.glassfish.hk2:hk2-locator:2.5.0-b32",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:1.3.2",
-                    "javax.ws.rs:javax.ws.rs-api:2.0.1",
-                    "javax.inject:javax.inject:1",
-                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
+                    "javax.ws.rs:javax.ws.rs-api:2.0.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar"
                 ],
-                "sha256": "4df653fc69d5feec7ad1928018f964e12a7513bcea7b5e8b1aa4b1f5a815815f"
+                "sha256": "4df653fc69d5feec7ad1928018f964e12a7513bcea7b5e8b1aa4b1f5a815815f",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
+                    "javax.inject:javax.inject:jar:sources:1",
                     "org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:sources:2.25.1",
+                    "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
+                    "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
+                    "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
-                    "javax.inject:javax.inject:jar:sources:1",
                     "org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:sources:2.25.1",
-                    "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                    "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
-                    "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar"
                 ],
-                "sha256": "9c575917a1f73ddbf4d2e297bbaa882c3c1c2741b818c0870d9a4c9653cd7608"
+                "sha256": "9c575917a1f73ddbf4d2e297bbaa882c3c1c2741b818c0870d9a4c9653cd7608",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-common/2.25.1/jersey-common-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-server:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar",
-                "directDependencies": [
+                "dependencies": [
+                    "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
+                    "org.javassist:javassist:3.20.0-GA",
+                    "org.glassfish.jersey.bundles.repackaged:jersey-guava:2.25.1",
                     "org.glassfish.hk2:hk2-locator:2.5.0-b32",
                     "org.glassfish.jersey.media:jersey-media-jaxb:2.25.1",
                     "org.glassfish.jersey.core:jersey-client:2.25.1",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:1.3.2",
                     "javax.ws.rs:javax.ws.rs-api:2.0.1",
                     "javax.validation:validation-api:1.1.0.Final",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1"
+                    "javax.inject:javax.inject:1",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1",
+                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "dependencies": [
-                    "org.glassfish.hk2:osgi-resource-locator:1.0.1",
+                "directDependencies": [
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                    "org.javassist:javassist:3.20.0-GA",
-                    "org.glassfish.jersey.bundles.repackaged:jersey-guava:2.25.1",
                     "org.glassfish.hk2:hk2-locator:2.5.0-b32",
                     "org.glassfish.jersey.media:jersey-media-jaxb:2.25.1",
                     "org.glassfish.jersey.core:jersey-client:2.25.1",
                     "org.glassfish.hk2:hk2-api:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-utils:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:1.3.2",
                     "javax.ws.rs:javax.ws.rs-api:2.0.1",
                     "javax.validation:validation-api:1.1.0.Final",
-                    "javax.inject:javax.inject:1",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1",
-                    "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
+                    "org.glassfish.jersey.core:jersey-common:2.25.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar"
                 ],
-                "sha256": "4b9cdae8eae88b75762614b9a458f5aac47cf6486fe408206fc64e38b80469ae"
+                "sha256": "4b9cdae8eae88b75762614b9a458f5aac47cf6486fe408206fc64e38b80469ae",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.core:jersey-server:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "org.glassfish.jersey.media:jersey-media-jaxb:jar:sources:2.25.1",
+                    "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
+                    "javax.inject:javax.inject:jar:sources:1",
+                    "org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:sources:2.25.1",
                     "org.glassfish.jersey.core:jersey-client:jar:sources:2.25.1",
                     "javax.validation:validation-api:jar:sources:1.1.0.Final",
+                    "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
+                    "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
+                    "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "org.glassfish.jersey.media:jersey-media-jaxb:jar:sources:2.25.1",
-                    "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
-                    "javax.inject:javax.inject:jar:sources:1",
-                    "org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:sources:2.25.1",
                     "org.glassfish.jersey.core:jersey-client:jar:sources:2.25.1",
                     "javax.validation:validation-api:jar:sources:1.1.0.Final",
-                    "org.glassfish.hk2:hk2-utils:jar:sources:2.5.0-b32",
                     "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1",
-                    "org.javassist:javassist:jar:sources:3.20.0-GA",
                     "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
-                    "org.glassfish.hk2.external:aopalliance-repackaged:jar:sources:2.5.0-b32",
                     "javax.annotation:javax.annotation-api:jar:sources:1.3.2",
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar"
                 ],
-                "sha256": "14453a10d06bc8556f019536e72f2d1c9541f96da1d7c0bba2dce810df58960f"
+                "sha256": "14453a10d06bc8556f019536e72f2d1c9541f96da1d7c0bba2dce810df58960f",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/core/jersey-server/2.25.1/jersey-server-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.ext:jersey-entity-filtering:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar",
-                "directDependencies": [
+                "dependencies": [
                     "javax.ws.rs:javax.ws.rs-api:2.0.1"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "javax.ws.rs:javax.ws.rs-api:2.0.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar"
                 ],
-                "sha256": "34400c6cc739e2084401462c30f969189d1da4bb03b77a7153cd5791f79fbf12"
+                "sha256": "34400c6cc739e2084401462c30f969189d1da4bb03b77a7153cd5791f79fbf12",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.ext:jersey-entity-filtering:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar",
-                "directDependencies": [
+                "dependencies": [
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1"
                 ],
-                "dependencies": [
+                "directDependencies": [
                     "javax.ws.rs:javax.ws.rs-api:jar:sources:2.0.1"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar",
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar"
                 ],
-                "sha256": "9bc50a9ab5ce335f1274e38f63cf4eee31a3e766b2eddc0f62fc0969e60a1c76"
+                "sha256": "9bc50a9ab5ce335f1274e38f63cf4eee31a3e766b2eddc0f62fc0969e60a1c76",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/ext/jersey-entity-filtering/2.25.1/jersey-entity-filtering-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.media:jersey-media-jaxb:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar",
-                "directDependencies": [
-                    "org.glassfish.hk2:osgi-resource-locator:1.0.1",
-                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-locator:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-api:2.5.0-b32",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
@@ -7705,24 +7736,24 @@
                     "org.glassfish.jersey.core:jersey-common:2.25.1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar",
+                "directDependencies": [
+                    "org.glassfish.hk2:osgi-resource-locator:1.0.1",
+                    "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-locator:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-api:2.5.0-b32",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar"
                 ],
-                "sha256": "05526bed0ffc07c2cea6b399f4e61ae3c99e44021e28a4af926ed1d867ba3fbe"
+                "sha256": "05526bed0ffc07c2cea6b399f4e61ae3c99e44021e28a4af926ed1d867ba3fbe",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.media:jersey-media-jaxb:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
-                "directDependencies": [
-                    "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
-                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
-                    "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
-                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
-                    "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
                     "javax.inject:javax.inject:jar:sources:1",
@@ -7737,24 +7768,24 @@
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
-                "mirror_urls": [
-                    "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
-                    "https://maven.google.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
-                    "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar"
-                ],
-                "sha256": "1a6b577d54f94ac7abbecf365f2ea52388c3299a00c82c32a5014d84dd1412bf"
-            },
-            {
-                "coord": "org.glassfish.jersey.media:jersey-media-json-jackson:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar",
                 "directDependencies": [
-                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
-                    "org.glassfish.jersey.ext:jersey-entity-filtering:2.25.1",
-                    "org.glassfish.jersey.core:jersey-common:2.25.1",
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.8.8"
+                    "org.glassfish.hk2:osgi-resource-locator:jar:sources:1.0.1",
+                    "org.glassfish.hk2.external:javax.inject:jar:sources:2.5.0-b32",
+                    "org.glassfish.hk2:hk2-locator:jar:sources:2.5.0-b32",
+                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
+                    "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
+                "mirror_urls": [
+                    "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
+                    "https://maven.google.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar",
+                    "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar"
                 ],
+                "sha256": "1a6b577d54f94ac7abbecf365f2ea52388c3299a00c82c32a5014d84dd1412bf",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-jaxb/2.25.1/jersey-media-jaxb-2.25.1-sources.jar"
+            },
+            {
+                "coord": "org.glassfish.jersey.media:jersey-media-json-jackson:2.25.1",
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
@@ -7776,24 +7807,24 @@
                     "com.fasterxml.jackson.core:jackson-core:2.8.8",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.core:jackson-annotations:2.8.8",
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.8.8",
+                    "org.glassfish.jersey.ext:jersey-entity-filtering:2.25.1",
+                    "org.glassfish.jersey.core:jersey-common:2.25.1",
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.8.8"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar"
                 ],
-                "sha256": "d449a6343389dd06f4440cc4da5b033e65d1197e4193ee7fea737e2ba5c9babe"
+                "sha256": "d449a6343389dd06f4440cc4da5b033e65d1197e4193ee7fea737e2ba5c9babe",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.media:jersey-media-json-jackson:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar",
-                "directDependencies": [
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
-                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8",
-                    "org.glassfish.jersey.ext:jersey-entity-filtering:jar:sources:2.25.1",
-                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1"
-                ],
                 "dependencies": [
                     "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar:sources:2.8.8",
                     "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:jar:sources:2.8.8",
@@ -7815,21 +7846,24 @@
                     "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1",
                     "org.glassfish.hk2:hk2-api:jar:sources:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar",
+                "directDependencies": [
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.core:jackson-annotations:jar:sources:2.8.8",
+                    "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar:sources:2.8.8",
+                    "org.glassfish.jersey.ext:jersey-entity-filtering:jar:sources:2.25.1",
+                    "org.glassfish.jersey.core:jersey-common:jar:sources:2.25.1"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar",
                     "https://maven.google.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar"
                 ],
-                "sha256": "551989f74efca4358534416bc3a6e726e1270894ea1965d0d30c3e475fae3836"
+                "sha256": "551989f74efca4358534416bc3a6e726e1270894ea1965d0d30c3e475fae3836",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-json-jackson/2.25.1/jersey-media-json-jackson-2.25.1-sources.jar"
             },
             {
                 "coord": "org.glassfish.jersey.media:jersey-media-multipart:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar",
-                "directDependencies": [
-                    "org.glassfish.jersey.core:jersey-common:2.25.1",
-                    "org.jvnet.mimepull:mimepull:1.9.7"
-                ],
                 "dependencies": [
                     "org.glassfish.hk2:osgi-resource-locator:1.0.1",
                     "org.glassfish.hk2.external:javax.inject:2.5.0-b32",
@@ -7845,21 +7879,21 @@
                     "org.glassfish.jersey.core:jersey-common:2.25.1",
                     "org.glassfish.hk2.external:aopalliance-repackaged:2.5.0-b32"
                 ],
-                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar",
+                "directDependencies": [
+                    "org.glassfish.jersey.core:jersey-common:2.25.1",
+                    "org.jvnet.mimepull:mimepull:1.9.7"
+                ],
+                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar",
                 "mirror_urls": [
                     "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar",
                     "https://maven.google.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar",
                     "https://repo1.maven.org/maven2/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar"
                 ],
-                "sha256": "909b669f76b8883a9218fb0fbc5022a286ead7d17b29aafa532b31f19ab4afcc"
+                "sha256": "909b669f76b8883a9218fb0fbc5022a286ead7d17b29aafa532b31f19ab4afcc",
+                "url": "https://jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1.jar"
             },
             {
                 "coord": "org.glassfish.jersey.media:jersey-media-multipart:jar:sources:2.25.1",
-                "file": "v1/https/jcenter.bintray.com/org/glassfish/jersey/media/jersey-media-multipart/2.25.1/jersey-media-multipart-2.25.1-sources.jar",
... 1604 lines suppressed ...