Merge remote-tracking branch 'origin/master' into HDDS-2665-ofs
diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml
index fc2dda1..4e63f41 100644
--- a/.github/workflows/post-commit.yml
+++ b/.github/workflows/post-commit.yml
@@ -247,6 +247,12 @@
          env:
            SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }}
            GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
+       - name: Upload coverage to Codecov
+         uses: codecov/codecov-action@v1
+         with:
+           file: ./target/coverage/all.xml
+           name: codecov-umbrella
+           fail_ci_if_error: true
        - uses: actions/upload-artifact@master
          with:
            name: coverage
diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml
index f9a4230..770aab2 100644
--- a/.github/workflows/pr.yml
+++ b/.github/workflows/pr.yml
@@ -219,4 +219,34 @@
           if: always()
           with:
             name: it-ozone
-            path: mnt/ozone/target/integration
\ No newline at end of file
+            path: mnt/ozone/target/integration
+  coverage:
+    name: coverage
+    runs-on: ubuntu-18.04
+    needs:
+      - it-hdds-om
+      - it-ozone
+      - it-client
+      - it-filesystem-contract
+      - it-filesystem
+      - it-freon
+      - unit
+    steps:
+      - uses: actions/checkout@v2
+      - uses: ./.github/buildenv
+        with:
+          args: ./hadoop-ozone/dev-support/checks/build.sh
+      - uses: actions/download-artifact@v2
+        with:
+          path: target/artifacts
+      - run: ./.github/coverage-report.sh
+      - name: Upload coverage to Codecov
+        uses: codecov/codecov-action@v1
+        with:
+          file: ./target/coverage/all.xml
+          name: codecov-umbrella
+          fail_ci_if_error: true
+      - uses: actions/upload-artifact@master
+        with:
+          name: coverage
+          path: target/coverage
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index ff83dff..28d4491 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -51,7 +51,6 @@
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.util.Time;
 import java.util.concurrent.TimeoutException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -434,7 +433,7 @@
     final CompletableFuture<ContainerCommandResponseProto> replyFuture =
         new CompletableFuture<>();
     semaphore.acquire();
-    long requestTime = Time.monotonicNowNanos();
+    long requestTime = System.nanoTime();
     metrics.incrPendingContainerOpsMetrics(request.getCmdType());
     // create a new grpc stream for each non-async call.
 
@@ -449,7 +448,7 @@
                 replyFuture.complete(value);
                 metrics.decrPendingContainerOpsMetrics(request.getCmdType());
                 metrics.addContainerOpsLatency(request.getCmdType(),
-                    Time.monotonicNowNanos() - requestTime);
+                    System.nanoTime() - requestTime);
                 semaphore.release();
               }
 
@@ -458,7 +457,7 @@
                 replyFuture.completeExceptionally(t);
                 metrics.decrPendingContainerOpsMetrics(request.getCmdType());
                 metrics.addContainerOpsLatency(request.getCmdType(),
-                    Time.monotonicNowNanos() - requestTime);
+                    System.nanoTime() - requestTime);
                 semaphore.release();
               }
 
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
index 7ff5ab1..2fa3342 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
@@ -47,7 +47,6 @@
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
-import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -301,7 +300,7 @@
   public XceiverClientReply sendCommandAsync(
       ContainerCommandRequestProto request) {
     XceiverClientReply asyncReply = new XceiverClientReply(null);
-    long requestTime = Time.monotonicNowNanos();
+    long requestTime = System.nanoTime();
     CompletableFuture<RaftClientReply> raftClientReply =
         sendRequestAsync(request);
     metrics.incrPendingContainerOpsMetrics(request.getCmdType());
@@ -315,7 +314,7 @@
           }
           metrics.decrPendingContainerOpsMetrics(request.getCmdType());
           metrics.addContainerOpsLatency(request.getCmdType(),
-              Time.monotonicNowNanos() - requestTime);
+              System.nanoTime() - requestTime);
         }).thenApply(reply -> {
           try {
             if (!reply.isSuccess()) {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
index 7776b21..4607ce5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
@@ -29,13 +29,13 @@
 import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
-import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.slf4j.Logger;
@@ -59,6 +59,7 @@
   private ThreadLocal<List<DatanodeDetails>> nodesInOrder = new ThreadLocal<>();
   // Current reported Leader for the pipeline
   private UUID leaderId;
+  private String strLeaderID = "";
   // Timestamp for pipeline upon creation
   private Instant creationTimestamp;
 
@@ -146,6 +147,9 @@
    */
   void setLeaderId(UUID leaderId) {
     this.leaderId = leaderId;
+    if (leaderId != null) {
+      this.strLeaderID = leaderId.toString();
+    }
   }
 
   /**
@@ -256,16 +260,19 @@
 
   public HddsProtos.Pipeline getProtobufMessage()
       throws UnknownPipelineStateException {
+    List<HddsProtos.DatanodeDetailsProto> members = new ArrayList<>();
+    for (DatanodeDetails dn : nodeStatus.keySet()) {
+      members.add(dn.getProtoBufMessage());
+    }
+
     HddsProtos.Pipeline.Builder builder = HddsProtos.Pipeline.newBuilder()
         .setId(id.getProtobuf())
         .setType(type)
         .setFactor(factor)
         .setState(PipelineState.getProtobuf(state))
-        .setLeaderID(leaderId != null ? leaderId.toString() : "")
+        .setLeaderID(strLeaderID)
         .setCreationTimeStamp(creationTimestamp.toEpochMilli())
-        .addAllMembers(nodeStatus.keySet().stream()
-            .map(DatanodeDetails::getProtoBufMessage)
-            .collect(Collectors.toList()));
+        .addAllMembers(members);
     // To save the message size on wire, only transfer the node order based on
     // network topology
     List<DatanodeDetails> nodes = nodesInOrder.get();
@@ -289,14 +296,19 @@
   public static Pipeline getFromProtobuf(HddsProtos.Pipeline pipeline)
       throws UnknownPipelineStateException {
     Preconditions.checkNotNull(pipeline, "Pipeline is null");
+
+    List<DatanodeDetails> nodes = new ArrayList<>();
+    for (DatanodeDetailsProto member : pipeline.getMembersList()) {
+      nodes.add(DatanodeDetails.getFromProtoBuf(member));
+    }
+
     return new Builder().setId(PipelineID.getFromProtobuf(pipeline.getId()))
         .setFactor(pipeline.getFactor())
         .setType(pipeline.getType())
         .setState(PipelineState.fromProtobuf(pipeline.getState()))
         .setLeaderId(StringUtils.isNotEmpty(pipeline.getLeaderID()) ?
             UUID.fromString(pipeline.getLeaderID()) : null)
-        .setNodes(pipeline.getMembersList().stream()
-            .map(DatanodeDetails::getFromProtoBuf).collect(Collectors.toList()))
+        .setNodes(nodes)
         .setNodesInOrder(pipeline.getMemberOrdersList())
         .setCreateTimestamp(pipeline.getCreationTimeStamp())
         .build();
@@ -341,7 +353,7 @@
     b.append(", Type:").append(getType());
     b.append(", Factor:").append(getFactor());
     b.append(", State:").append(getPipelineState());
-    b.append(", leaderId:").append(getLeaderId());
+    b.append(", leaderId:").append(strLeaderID);
     b.append(", CreationTimestamp").append(getCreationTimestamp());
     b.append("]");
     return b.toString();
diff --git a/hadoop-hdds/docs/content/design/ozone-volume-management.md b/hadoop-hdds/docs/content/design/ozone-volume-management.md
index 6c63656..c996a87 100644
--- a/hadoop-hdds/docs/content/design/ozone-volume-management.md
+++ b/hadoop-hdds/docs/content/design/ozone-volume-management.md
@@ -4,7 +4,7 @@
 date: 2020-04-02
 jira: HDDS-3331
 status: accepted
-author: Marton Elek, Arpit Agarwall, Sunjay Radia
+author: Marton Elek, Arpit Agarwal, Sanjay Radia
 ---
 
 <!--
@@ -28,7 +28,7 @@
 
 ## The Problems
 
- 1. Unpriviliged users cannot enumerate volumes.
+ 1. Unprivileged users cannot enumerate volumes.
  2. The mapping of S3 buckets to Ozone volumes is confusing. Based on external feedback it's hard to understand the exact Ozone URL to be used.
  3. The volume name is not friendly and cannot be remembered by humans.
  4. Ozone buckets created via the native object store interface are not visible via the S3 gateway.
@@ -89,7 +89,7 @@
 
 ### Solving the mapping problem (#2-4 from the problem listing)
 
- 1. Let's always use `s3` volume for all the s3 buckets **if the bucket is created from the s3 interface**.
+ 1. Let's always use `s3v` volume for all the s3 buckets **if the bucket is created from the s3 interface**.
 
 This is an easy an fast method, but with this approach not all the volumes are avilable via the S3 interface. We need to provide a method to publish any of the ozone volumes / buckets.
 
@@ -102,23 +102,34 @@
  To implement the second (expose ozone buckets as s3 buckets) we have multiple options:
 
    1. Store some metadata (** s3 bucket name **) on each of the buckets
-   2. Implement a **bind mount** mechanic which makes it possible to *mount* any volume/buckets to the specific "s3" volume.
+   2. Implement a **symbolic link** mechanism which makes it possible to *link* to any volume/buckets from the "s3" volume.
 
 The first approach required a secondary cache table and it violates the naming hierarchy. The s3 bucket name is a global unique name, therefore it's more than just a single attribute on a specific object. It's more like an element in the hierachy. For this reason the second option is proposed:
 
-For example if the default s3 volume is `s3`
+For example if the default s3 volume is `s3v`
 
- 1. Every new buckets created via s3 interface will be placed under the `/s3` volume
- 2. Any existing **Ozone** buckets can be exposed with mounting it to s3: `ozone sh mount /vol1/bucket1 /s3/s3bucketname`
+ 1. Every new buckets created via s3 interface will be placed under the `/s3v` volume
+ 2. Any existing **Ozone** buckets can be exposed by linking to it from s3: `ozone sh bucket link /vol1/bucket1 /s3v/s3bucketname`
 
 **Lock contention problem**
 
-One possible problem with using just one volume is using the locks of the same volume for all the D3 buckets (thanks Xiaoyu). But this shouldn't be a big problem.
+One possible problem with using just one volume is using the locks of the same volume for all the S3 buckets (thanks Xiaoyu). But this shouldn't be a big problem.
 
  1. We hold only a READ lock. Most of the time it can acquired without any contention (writing lock is required only to change owner / set quota)
- 2. For symbolic link / bind mounts the read lock is only required for the first read. After that the lock of the referenced volume will be used. In case of any performance problem multiple volumes + bind mounts can be used.
+ 2. For symbolic link the read lock is only required for the first read. After that the lock of the referenced volume will be used. In case of any performance problem multiple volumes and links can be used.
 
-Note: Sunjay is added to the authors as the original proposal of this approach.
+Note: Sanjay is added to the authors as the original proposal of this approach.
+
+#### Implementation details
+
+ * `bucket link` operation creates a link bucket.  Links are like regular buckets, stored in DB the same way, but with two new, optional pieces of information: source volume and bucket.  (The bucket being referenced by the link is called "source", not "target", to follow symlink terminology.)
+ * Link buckets share the namespace with regular buckets.  If a bucket or link with the same name already exists, a `BUCKET_ALREADY_EXISTS` result is returned.
+ * Link buckets are not inherently specific to a user, access is restricted only by ACL.
+ * Links are persistent, ie. they can be used until they are deleted.
+ * Existing bucket operations (info, delete, ACL) work on the link object in the same way as they do on regular buckets.  No new link-specific RPC is required.
+ * Links are followed for key operations (list, get, put, etc.).  Read permission on the link is required for this.
+ * Checks for existence of the source bucket, as well as ACL, are performed only when following the link (similar to symlinks).  Source bucket is not checked when operating on the link bucket itself (eg. deleting it).  This avoids the need for reverse checks for each bucket delete or ACL change.
+ * Bucket links are generic, not restricted to the `s3v` volume.
 
 ## Alternative approaches and reasons to reject
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
index 3c7af08..69fbc08 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
@@ -33,7 +33,6 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.Predicate;
-import java.util.stream.Collectors;
 
 /**
  * Holds the data structures which maintain the information about pipeline and
@@ -152,9 +151,14 @@
   List<Pipeline> getPipelines(ReplicationType type) {
     Preconditions.checkNotNull(type, "Replication type cannot be null");
 
-    return pipelineMap.values().stream()
-        .filter(p -> p.getType().equals(type))
-        .collect(Collectors.toList());
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (Pipeline pipeline : pipelineMap.values()) {
+      if (pipeline.getType() == type) {
+        pipelines.add(pipeline);
+      }
+    }
+
+    return pipelines;
   }
 
   /**
@@ -168,10 +172,14 @@
     Preconditions.checkNotNull(type, "Replication type cannot be null");
     Preconditions.checkNotNull(factor, "Replication factor cannot be null");
 
-    return pipelineMap.values().stream()
-        .filter(pipeline -> pipeline.getType() == type
-            && pipeline.getFactor() == factor)
-        .collect(Collectors.toList());
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (Pipeline pipeline : pipelineMap.values()) {
+      if (pipeline.getType() == type && pipeline.getFactor() == factor) {
+        pipelines.add(pipeline);
+      }
+    }
+
+    return pipelines;
   }
 
   /**
@@ -188,10 +196,16 @@
 
     Set<PipelineState> pipelineStates = new HashSet<>();
     pipelineStates.addAll(Arrays.asList(states));
-    return pipelineMap.values().stream().filter(
-        pipeline -> pipeline.getType() == type && pipelineStates
-            .contains(pipeline.getPipelineState()))
-        .collect(Collectors.toList());
+
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (Pipeline pipeline : pipelineMap.values()) {
+      if (pipeline.getType() == type
+          && pipelineStates.contains(pipeline.getPipelineState())) {
+        pipelines.add(pipeline);
+      }
+    }
+
+    return pipelines;
   }
 
   /**
@@ -210,15 +224,21 @@
     Preconditions.checkNotNull(state, "Pipeline state cannot be null");
 
     if (state == PipelineState.OPEN) {
-      return Collections.unmodifiableList(
+      return new ArrayList<>(
           query2OpenPipelines.getOrDefault(
               new PipelineQuery(type, factor), Collections.EMPTY_LIST));
     }
-    return pipelineMap.values().stream().filter(
-        pipeline -> pipeline.getType() == type
-            && pipeline.getPipelineState() == state
-            && pipeline.getFactor() == factor)
-        .collect(Collectors.toList());
+
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (Pipeline pipeline : pipelineMap.values()) {
+      if (pipeline.getType() == type
+          && pipeline.getPipelineState() == state
+          && pipeline.getFactor() == factor) {
+        pipelines.add(pipeline);
+      }
+    }
+
+    return pipelines;
   }
 
   /**
@@ -242,10 +262,18 @@
         .checkNotNull(excludeDns, "Datanode exclude list cannot be null");
     Preconditions
         .checkNotNull(excludeDns, "Pipeline exclude list cannot be null");
-    return getPipelines(type, factor, state).stream().filter(
-        pipeline -> !discardPipeline(pipeline, excludePipelines)
-            && !discardDatanode(pipeline, excludeDns))
-        .collect(Collectors.toList());
+
+    List<Pipeline> pipelines = getPipelines(type, factor, state);
+    Iterator<Pipeline> iter = pipelines.iterator();
+    while (iter.hasNext()) {
+      Pipeline pipeline = iter.next();
+      if (discardPipeline(pipeline, excludePipelines) ||
+          discardDatanode(pipeline, excludeDns)) {
+        iter.remove();
+      }
+    }
+
+    return pipelines;
   }
 
   private boolean discardPipeline(Pipeline pipeline,
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index e049ef5..d935f24 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -285,10 +285,13 @@
 
     List<ContainerWithPipeline> cpList = new ArrayList<>();
 
+    StringBuilder strContainerIDs = new StringBuilder();
     for (Long containerID : containerIDs) {
       try {
         ContainerWithPipeline cp = getContainerWithPipelineCommon(containerID);
         cpList.add(cp);
+        strContainerIDs.append(ContainerID.valueof(containerID).toString());
+        strContainerIDs.append(",");
       } catch (IOException ex) {
         AUDIT.logReadFailure(buildAuditMessageForFailure(
             SCMAction.GET_CONTAINER_WITH_PIPELINE_BATCH,
@@ -298,11 +301,10 @@
       }
     }
 
+
     AUDIT.logReadSuccess(buildAuditMessageForSuccess(
         SCMAction.GET_CONTAINER_WITH_PIPELINE_BATCH,
-        Collections.singletonMap("containerIDs",
-        containerIDs.stream().map(id -> ContainerID.valueof(id).toString())
-            .collect(Collectors.joining(",")))));
+        Collections.singletonMap("containerIDs", strContainerIDs.toString())));
 
     return cpList;
   }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 4a2d713..8437699 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -477,7 +477,7 @@
    * @return listOfAcls
    * */
   private List<OzoneAcl> getAclList() {
-    return OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroups(),
+    return OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
         userRights, groupRights);
   }
 
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
index ee71fb2..783089e 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
@@ -24,12 +24,12 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.stream.Collectors;
 
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 import org.apache.hadoop.util.Time;
 
@@ -384,6 +384,12 @@
   public KeyInfo getProtobuf() {
     long latestVersion = keyLocationVersions.size() == 0 ? -1 :
         keyLocationVersions.get(keyLocationVersions.size() - 1).getVersion();
+
+    List<KeyLocationList> keyLocations = new ArrayList<>();
+    for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+      keyLocations.add(locationInfoGroup.getProtobuf());
+    }
+
     KeyInfo.Builder kb = KeyInfo.newBuilder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
@@ -391,9 +397,7 @@
         .setDataSize(dataSize)
         .setFactor(factor)
         .setType(type)
-        .addAllKeyLocationList(keyLocationVersions.stream()
-            .map(OmKeyLocationInfoGroup::getProtobuf)
-            .collect(Collectors.toList()))
+        .addAllKeyLocationList(keyLocations)
         .setLatestVersion(latestVersion)
         .setCreationTime(creationTime)
         .setModificationTime(modificationTime)
@@ -411,13 +415,18 @@
     if (keyInfo == null) {
       return null;
     }
+
+    List<OmKeyLocationInfoGroup> omKeyLocationInfos = new ArrayList<>();
+    for (KeyLocationList keyLocationList : keyInfo.getKeyLocationListList()) {
+      omKeyLocationInfos.add(
+          OmKeyLocationInfoGroup.getFromProtobuf(keyLocationList));
+    }
+
     Builder builder = new Builder()
         .setVolumeName(keyInfo.getVolumeName())
         .setBucketName(keyInfo.getBucketName())
         .setKeyName(keyInfo.getKeyName())
-        .setOmKeyLocationInfos(keyInfo.getKeyLocationListList().stream()
-            .map(OmKeyLocationInfoGroup::getFromProtobuf)
-            .collect(Collectors.toList()))
+        .setOmKeyLocationInfos(omKeyLocationInfos)
         .setDataSize(keyInfo.getDataSize())
         .setCreationTime(keyInfo.getCreationTime())
         .setModificationTime(keyInfo.getModificationTime())
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
index 8bdcee3..bd97757 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.ozone.om.helpers;
 
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList;
 
 import java.io.IOException;
@@ -68,11 +69,14 @@
 
   public static OmKeyLocationInfoGroup getFromProtobuf(
       KeyLocationList keyLocationList) {
+    List<OmKeyLocationInfo> locations = new ArrayList<>();
+    for (KeyLocation keyLocation : keyLocationList
+        .getKeyLocationsList()) {
+      locations.add(OmKeyLocationInfo.getFromProtobuf(keyLocation));
+    }
+
     return new OmKeyLocationInfoGroup(
-        keyLocationList.getVersion(),
-        keyLocationList.getKeyLocationsList().stream()
-            .map(OmKeyLocationInfo::getFromProtobuf)
-            .collect(Collectors.toList()));
+        keyLocationList.getVersion(), locations);
   }
 
   /**
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
index fd42fea..5b099d8 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
@@ -26,6 +26,7 @@
 import org.apache.hadoop.ozone.security.acl.RequestContext;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -53,7 +54,7 @@
    * @return list of OzoneAcls
    * */
   public static List<OzoneAcl> getAclList(String userName,
-      List<String> userGroups, ACLType userRights, ACLType groupRights) {
+      String[] userGroups, ACLType userRights, ACLType groupRights) {
 
     List<OzoneAcl> listOfAcls = new ArrayList<>();
 
@@ -61,7 +62,7 @@
     listOfAcls.add(new OzoneAcl(USER, userName, userRights, ACCESS));
     if(userGroups != null) {
       // Group ACLs of the User.
-      userGroups.forEach((group) -> listOfAcls.add(
+      Arrays.asList(userGroups).forEach((group) -> listOfAcls.add(
           new OzoneAcl(GROUP, group, groupRights, ACCESS)));
     }
     return listOfAcls;
@@ -190,8 +191,11 @@
    * @return list of OzoneAcl.
    */
   public static List<OzoneAcl> fromProtobuf(List<OzoneAclInfo> protoAcls) {
-    return protoAcls.stream().map(acl->OzoneAcl.fromProtobuf(acl))
-        .collect(Collectors.toList());
+    List<OzoneAcl> ozoneAcls = new ArrayList<>();
+    for (OzoneAclInfo aclInfo : protoAcls) {
+      ozoneAcls.add(OzoneAcl.fromProtobuf(aclInfo));
+    }
+    return ozoneAcls;
   }
 
   /**
@@ -200,8 +204,11 @@
    * @return list of OzoneAclInfo.
    */
   public static List<OzoneAclInfo> toProtobuf(List<OzoneAcl> protoAcls) {
-    return protoAcls.stream().map(acl->OzoneAcl.toProtobuf(acl))
-        .collect(Collectors.toList());
+    List<OzoneAclInfo> ozoneAclInfos = new ArrayList<>();
+    for (OzoneAcl acl : protoAcls) {
+      ozoneAclInfos.add(OzoneAcl.toProtobuf(acl));
+    }
+    return ozoneAclInfos;
   }
 
   /**
diff --git a/hadoop-ozone/dev-support/checks/integration.sh b/hadoop-ozone/dev-support/checks/integration.sh
index db5bf23..e2bf3dc 100755
--- a/hadoop-ozone/dev-support/checks/integration.sh
+++ b/hadoop-ozone/dev-support/checks/integration.sh
@@ -23,8 +23,8 @@
 mkdir -p "$REPORT_DIR"
 
 export MAVEN_OPTS="-Xmx4096m"
-mvn -B install -DskipTests
-mvn -B -fae test -pl :hadoop-ozone-integration-test,:mini-chaos-tests "$@" \
+mvn -B install -DskipTests -Dskip.yarn
+mvn -B -fae test -Dskip.yarn -pl :hadoop-ozone-integration-test,:mini-chaos-tests "$@" \
   | tee "${REPORT_DIR}/output.log"
 rc=$?
 
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
index e108203..7a1cb5b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
@@ -279,7 +279,7 @@
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName(KEY_NAME)
         .setDataSize(1000)
-        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroups(),
+        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
             ALL, ALL))
         .build();
     LambdaTestUtils.intercept(OMException.class,
@@ -1363,7 +1363,7 @@
         .setFactor(ReplicationFactor.ONE)
         .setDataSize(0)
         .setType(ReplicationType.STAND_ALONE)
-        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroups(),
+        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
             ALL, ALL))
         .setVolumeName(VOLUME_NAME);
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 6f7e853..9f556f8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -2027,7 +2027,8 @@
    * Allocate a key.
    *
    * @param args - attributes of the key.
-   * @return OmKeyInfo - the info about the allocated key.
+   * @return OpenKeySession - a handler to key that client uses to
+   * talk to container.
    * @throws IOException
    */
   @Override
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
index 8a99e4f..63467b7 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
@@ -19,9 +19,9 @@
 package org.apache.hadoop.ozone.om.request.key;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
@@ -47,6 +47,8 @@
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyLocation;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .OMResponse;
@@ -132,10 +134,10 @@
           keyName, IAccessAuthorizer.ACLType.WRITE,
           commitKeyRequest.getClientID());
 
-      List<OmKeyLocationInfo> locationInfoList = commitKeyArgs
-          .getKeyLocationsList().stream()
-          .map(OmKeyLocationInfo::getFromProtobuf)
-          .collect(Collectors.toList());
+      List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+      for (KeyLocation keyLocation : commitKeyArgs.getKeyLocationsList()) {
+        locationInfoList.add(OmKeyLocationInfo.getFromProtobuf(keyLocation));
+      }
 
       bucketLockAcquired = omMetadataManager.getLock().acquireLock(BUCKET_LOCK,
           volumeName, bucketName);
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneNativeAuthorizer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneNativeAuthorizer.java
index ac1ad20..219ec97 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneNativeAuthorizer.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneNativeAuthorizer.java
@@ -176,7 +176,7 @@
         .setDataSize(0)
         .setType(HddsProtos.ReplicationType.STAND_ALONE)
         .setAcls(OzoneAclUtil.getAclList(testUgi.getUserName(),
-            testUgi.getGroups(), ALL, ALL))
+            testUgi.getGroupNames(), ALL, ALL))
         .build();
 
     if (keyName.split(OZONE_URI_DELIMITER).length > 1) {
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
index 7e22195..defe3c1 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
@@ -92,18 +92,7 @@
    */
   public BasicOzoneClientAdapterImpl(String volumeStr, String bucketStr)
       throws IOException {
-    this(createConf(), volumeStr, bucketStr);
-  }
-
-  private static OzoneConfiguration createConf() {
-    ClassLoader contextClassLoader =
-        Thread.currentThread().getContextClassLoader();
-    Thread.currentThread().setContextClassLoader(null);
-    try {
-      return new OzoneConfiguration();
-    } finally {
-      Thread.currentThread().setContextClassLoader(contextClassLoader);
-    }
+    this(new OzoneConfiguration(), volumeStr, bucketStr);
   }
 
   public BasicOzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
@@ -116,72 +105,63 @@
       ConfigurationSource hadoopConf, String volumeStr, String bucketStr)
       throws IOException {
 
-    ClassLoader contextClassLoader =
-        Thread.currentThread().getContextClassLoader();
-    Thread.currentThread().setContextClassLoader(null);
+    OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf);
+    if (omHost == null && OmUtils.isServiceIdsDefined(conf)) {
+      // When the host name or service id isn't given
+      // but ozone.om.service.ids is defined, declare failure.
 
-    try {
-      OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf);
-      if (omHost == null && OmUtils.isServiceIdsDefined(conf)) {
-        // When the host name or service id isn't given
-        // but ozone.om.service.ids is defined, declare failure.
-
-        // This is a safety precaution that prevents the client from
-        // accidentally failing over to an unintended OM.
-        throw new IllegalArgumentException("Service ID or host name must not"
-            + " be omitted when ozone.om.service.ids is defined.");
-      }
-
-      if (omPort != -1) {
-        // When the port number is specified, perform the following check
-        if (OmUtils.isOmHAServiceId(conf, omHost)) {
-          // If omHost is a service id, it shouldn't use a port
-          throw new IllegalArgumentException("Port " + omPort +
-              " specified in URI but host '" + omHost + "' is a "
-              + "logical (HA) OzoneManager and does not use port information.");
-        }
-      } else {
-        // When port number is not specified, read it from config
-        omPort = OmUtils.getOmRpcPort(conf);
-      }
-
-      SecurityConfig secConfig = new SecurityConfig(conf);
-
-      if (secConfig.isSecurityEnabled()) {
-        this.securityEnabled = true;
-      }
-
-      String replicationTypeConf =
-          conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
-              OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT);
-
-      int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
-          OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);
-
-      if (OmUtils.isOmHAServiceId(conf, omHost)) {
-        // omHost is listed as one of the service ids in the config,
-        // thus we should treat omHost as omServiceId
-        this.ozoneClient =
-            OzoneClientFactory.getRpcClient(omHost, conf);
-      } else if (StringUtils.isNotEmpty(omHost) && omPort != -1) {
-        this.ozoneClient =
-            OzoneClientFactory.getRpcClient(omHost, omPort, conf);
-      } else {
-        this.ozoneClient =
-            OzoneClientFactory.getRpcClient(conf);
-      }
-      objectStore = ozoneClient.getObjectStore();
-      this.volume = objectStore.getVolume(volumeStr);
-      this.bucket = volume.getBucket(bucketStr);
-      this.replicationType = ReplicationType.valueOf(replicationTypeConf);
-      this.replicationFactor = ReplicationFactor.valueOf(replicationCountConf);
-      this.configuredDnPort = conf.getInt(
-          OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
-    } finally {
-      Thread.currentThread().setContextClassLoader(contextClassLoader);
+      // This is a safety precaution that prevents the client from
+      // accidentally failing over to an unintended OM.
+      throw new IllegalArgumentException("Service ID or host name must not"
+          + " be omitted when ozone.om.service.ids is defined.");
     }
 
+    if (omPort != -1) {
+      // When the port number is specified, perform the following check
+      if (OmUtils.isOmHAServiceId(conf, omHost)) {
+        // If omHost is a service id, it shouldn't use a port
+        throw new IllegalArgumentException("Port " + omPort +
+            " specified in URI but host '" + omHost + "' is a "
+            + "logical (HA) OzoneManager and does not use port information.");
+      }
+    } else {
+      // When port number is not specified, read it from config
+      omPort = OmUtils.getOmRpcPort(conf);
+    }
+
+    SecurityConfig secConfig = new SecurityConfig(conf);
+
+    if (secConfig.isSecurityEnabled()) {
+      this.securityEnabled = true;
+    }
+
+    String replicationTypeConf =
+        conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
+            OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT);
+
+    int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
+        OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);
+
+    if (OmUtils.isOmHAServiceId(conf, omHost)) {
+      // omHost is listed as one of the service ids in the config,
+      // thus we should treat omHost as omServiceId
+      this.ozoneClient =
+          OzoneClientFactory.getRpcClient(omHost, conf);
+    } else if (StringUtils.isNotEmpty(omHost) && omPort != -1) {
+      this.ozoneClient =
+          OzoneClientFactory.getRpcClient(omHost, omPort, conf);
+    } else {
+      this.ozoneClient =
+          OzoneClientFactory.getRpcClient(conf);
+    }
+    objectStore = ozoneClient.getObjectStore();
+    this.volume = objectStore.getVolume(volumeStr);
+    this.bucket = volume.getBucket(bucketStr);
+    this.replicationType = ReplicationType.valueOf(replicationTypeConf);
+    this.replicationFactor = ReplicationFactor.valueOf(replicationCountConf);
+    this.configuredDnPort = conf.getInt(
+        OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
+        OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
   }
 
   @Override
diff --git a/hadoop-ozone/ozonefs-hadoop2/pom.xml b/hadoop-ozone/ozonefs-hadoop2/pom.xml
index 06ea2e2..8fcfec2 100644
--- a/hadoop-ozone/ozonefs-hadoop2/pom.xml
+++ b/hadoop-ozone/ozonefs-hadoop2/pom.xml
@@ -51,7 +51,7 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
-      <version>2.7.7</version>
+      <version>2.7.3</version>
     </dependency>
     <dependency>
       <groupId>log4j</groupId>
diff --git a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/Hadoop27RpcTransport.java b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/Hadoop27RpcTransport.java
index fc1c97f..76bd76c 100644
--- a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/Hadoop27RpcTransport.java
+++ b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/Hadoop27RpcTransport.java
@@ -20,6 +20,7 @@
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.io.Text;
@@ -58,7 +59,7 @@
         socket, UserGroupInformation.getCurrentUser(),
         ozoneConfiguration,
         NetUtils.getDefaultSocketFactory(ozoneConfiguration),
-        RPC.getRpcTimeout(ozoneConfiguration), null).getProxy();
+        getRpcTimeout(ozoneConfiguration), null).getProxy();
   }
 
   @Override
@@ -79,4 +80,8 @@
   public void close() throws IOException {
   }
 
+  private int getRpcTimeout(OzoneConfiguration conf) {
+    return conf.getInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY,
+        CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_DEFAULT);
+  }
 }
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/LICENSE b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/LICENSE
index 3a21949..2c38b3f 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/LICENSE
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/LICENSE
@@ -304,6 +304,32 @@
 
 -----
 
+The following software may be included in this product: @emotion/cache, @emotion/core, @emotion/css, @emotion/hash, @emotion/memoize, @emotion/serialize, @emotion/sheet, @emotion/stylis, @emotion/unitless, @emotion/utils, @emotion/weak-memoize, babel-plugin-emotion. A copy of the source code may be downloaded from https://github.com/emotion-js/emotion/tree/master/packages/cache (@emotion/cache), https://github.com/emotion-js/emotion/tree/master/packages/core (@emotion/core), https://github.com/emotion-js/emotion/tree/master/packages/css (@emotion/css), https://github.com/emotion-js/emotion/tree/master/packages/hash (@emotion/hash), https://github.com/emotion-js/emotion/tree/master/packages/memoize (@emotion/memoize), https://github.com/emotion-js/emotion/tree/master/packages/serialize (@emotion/serialize), https://github.com/emotion-js/emotion/tree/master/packages/sheet (@emotion/sheet), https://github.com/emotion-js/emotion/tree/master/packages/stylis (@emotion/stylis), https://github.com/emotion-js/emotion/tree/master/packages/unitless (@emotion/unitless), https://github.com/emotion-js/emotion/tree/master/packages/serialize (@emotion/utils), https://github.com/emotion-js/emotion/tree/master/packages/weak-memoize (@emotion/weak-memoize), https://github.com/emotion-js/emotion/tree/master/packages/babel-plugin-emotion (babel-plugin-emotion). This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) Emotion team and other contributors
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: @hapi/address. A copy of the source code may be downloaded from git://github.com/hapijs/address. This software contains the following license and notice below:
 
 Copyright (c) 2019, Project contributors
@@ -390,6 +416,167 @@
 
 -----
 
+The following software may be included in this product: @mapbox/geojson-rewind. A copy of the source code may be downloaded from git://github.com/mapbox/geojson-rewind.git. This software contains the following license and notice below:
+
+Copyright (c) 2020, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any
+purpose with or without fee is hereby granted, provided that the above
+copyright notice and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR
+OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+PERFORMANCE OF THIS SOFTWARE.
+
+-----
+
+The following software may be included in this product: @mapbox/geojson-types. This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2018 Mapbox
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: @mapbox/mapbox-gl-supported. A copy of the source code may be downloaded from git+https://github.com/mapbox/mapbox-gl-supported.git. This software contains the following license and notice below:
+
+BSD 3-Clause License
+
+Copyright (c) 2017, Mapbox
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the copyright holder nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: @mapbox/point-geometry. A copy of the source code may be downloaded from git@github.com:mapbox/point-geometry.git. This software contains the following license and notice below:
+
+Copyright (c) 2015, Mapbox <>
+
+Permission to use, copy, modify, and/or distribute this software for any
+purpose with or without fee is hereby granted, provided that the above
+copyright notice and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
+WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
+ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
+WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
+ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
+OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+
+-----
+
+The following software may be included in this product: @mapbox/tiny-sdf. A copy of the source code may be downloaded from git+https://github.com/mapbox/tiny-sdf.git. This software contains the following license and notice below:
+
+Copyright © 2016-2017 Mapbox, Inc.
+This code available under the terms of the BSD 2-Clause license.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: @mapbox/vector-tile. A copy of the source code may be downloaded from https://github.com/mapbox/vector-tile-js.git. This software contains the following license and notice below:
+
+Copyright (c) 2014, Mapbox
+
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+    * Neither the name of Mapbox nor the names of its contributors
+      may be used to endorse or promote products derived from this software
+      without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: @mapbox/whoots-js. A copy of the source code may be downloaded from https://github.com/mapbox/whoots-js.git. This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2017, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: @mrmlnc/readdir-enhanced. A copy of the source code may be downloaded from https://github.com/bigstickcarpet/readdir-enhanced.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -418,7 +605,65 @@
 
 -----
 
-The following software may be included in this product: @sindresorhus/is, aggregate-error, ansi-escapes, ansi-regex, ansi-styles, arrify, binary-extensions, boxen, callsites, camelcase, chalk, clean-stack, cli-boxes, cli-cursor, crypto-random-string, del, dot-prop, env-editor, escape-goat, escape-string-regexp, eslint-config-xo, eslint-config-xo-react, eslint-config-xo-typescript, eslint-formatter-pretty, eslint-plugin-ava, execa, find-up, get-stdin, get-stream, global-dirs, globals, globby, got, gzip-size, has-flag, has-yarn, import-fresh, import-local, import-modules, indent-string, internal-ip, invert-kv, irregular-plurals, is-absolute-url, is-docker, is-fullwidth-code-point, is-generator-fn, is-installed-globally, is-npm, is-obj, is-path-cwd, is-path-in-cwd, is-path-inside, is-root, is-svg, is-wsl, latest-version, lcid, leven, line-column-path, locate-path, log-symbols, lowercase-keys, make-dir, mem, meow, mimic-fn, mimic-response, normalize-url, onetime, open, open-editor, opn, os-locale, p-cancelable, p-is-promise, p-limit, p-locate, p-map, p-reduce, p-retry, p-try, package-json, parent-module, parse-json, parse-ms, path-exists, path-key, path-type, pify, pkg-dir, pkg-up, plur, prepend-http, pretty-bytes, pretty-ms, proto-props, pupa, quick-lru, read-pkg, read-pkg-up, registry-url, resolve-cwd, resolve-from, restore-cursor, semver-diff, shebang-regex, slash, string-length, string-width, strip-ansi, strip-json-comments, supports-color, term-size, to-readable-stream, type-fest, unique-string, url-parse-lax, widest-line, wrap-ansi, xdg-basedir. A copy of the source code may be downloaded from https://github.com/sindresorhus/is.git (@sindresorhus/is), https://github.com/sindresorhus/aggregate-error.git (aggregate-error), https://github.com/sindresorhus/ansi-escapes.git (ansi-escapes), https://github.com/chalk/ansi-regex.git (ansi-regex), https://github.com/chalk/ansi-styles.git (ansi-styles), https://github.com/sindresorhus/arrify.git (arrify), https://github.com/sindresorhus/binary-extensions.git (binary-extensions), https://github.com/sindresorhus/boxen.git (boxen), https://github.com/sindresorhus/callsites.git (callsites), https://github.com/sindresorhus/camelcase.git (camelcase), https://github.com/chalk/chalk.git (chalk), https://github.com/sindresorhus/clean-stack.git (clean-stack), https://github.com/sindresorhus/cli-boxes.git (cli-boxes), https://github.com/sindresorhus/cli-cursor.git (cli-cursor), https://github.com/sindresorhus/crypto-random-string.git (crypto-random-string), https://github.com/sindresorhus/del.git (del), https://github.com/sindresorhus/dot-prop.git (dot-prop), https://github.com/sindresorhus/env-editor.git (env-editor), https://github.com/sindresorhus/escape-goat.git (escape-goat), https://github.com/sindresorhus/escape-string-regexp.git (escape-string-regexp), https://github.com/xojs/eslint-config-xo.git (eslint-config-xo), https://github.com/xojs/eslint-config-xo-react.git (eslint-config-xo-react), https://github.com/xojs/eslint-config-xo-typescript.git (eslint-config-xo-typescript), https://github.com/sindresorhus/eslint-formatter-pretty.git (eslint-formatter-pretty), https://github.com/avajs/eslint-plugin-ava.git (eslint-plugin-ava), https://github.com/sindresorhus/execa.git (execa), https://github.com/sindresorhus/find-up.git (find-up), https://github.com/sindresorhus/get-stdin.git (get-stdin), https://github.com/sindresorhus/get-stream.git (get-stream), https://github.com/sindresorhus/global-dirs.git (global-dirs), https://github.com/sindresorhus/globals.git (globals), https://github.com/sindresorhus/globby.git (globby), https://github.com/sindresorhus/got.git (got), https://github.com/sindresorhus/gzip-size.git (gzip-size), https://github.com/sindresorhus/has-flag.git (has-flag), https://github.com/sindresorhus/has-yarn.git (has-yarn), https://github.com/sindresorhus/import-fresh.git (import-fresh), https://github.com/sindresorhus/import-local.git (import-local), https://github.com/sindresorhus/import-files.git (import-modules), https://github.com/sindresorhus/indent-string.git (indent-string), https://github.com/sindresorhus/internal-ip.git (internal-ip), https://github.com/sindresorhus/invert-kv.git (invert-kv), https://github.com/sindresorhus/irregular-plurals.git (irregular-plurals), https://github.com/sindresorhus/is-absolute-url.git (is-absolute-url), https://github.com/sindresorhus/is-docker.git (is-docker), https://github.com/sindresorhus/is-fullwidth-code-point.git (is-fullwidth-code-point), https://github.com/sindresorhus/is-generator-fn.git (is-generator-fn), https://github.com/sindresorhus/is-installed-globally.git (is-installed-globally), https://github.com/sindresorhus/is-npm.git (is-npm), https://github.com/sindresorhus/is-obj.git (is-obj), https://github.com/sindresorhus/is-path-cwd.git (is-path-cwd), https://github.com/sindresorhus/is-path-in-cwd.git (is-path-in-cwd), https://github.com/sindresorhus/is-path-inside.git (is-path-inside), https://github.com/sindresorhus/is-root.git (is-root), https://github.com/sindresorhus/is-svg.git (is-svg), https://github.com/sindresorhus/is-wsl.git (is-wsl), https://github.com/sindresorhus/latest-version.git (latest-version), https://github.com/sindresorhus/lcid.git (lcid), https://github.com/sindresorhus/leven.git (leven), https://github.com/sindresorhus/line-column-path.git (line-column-path), https://github.com/sindresorhus/locate-path.git (locate-path), https://github.com/sindresorhus/log-symbols.git (log-symbols), https://github.com/sindresorhus/lowercase-keys.git (lowercase-keys), https://github.com/sindresorhus/make-dir.git (make-dir), https://github.com/sindresorhus/mem.git (mem), https://github.com/sindresorhus/meow.git (meow), https://github.com/sindresorhus/mimic-fn.git (mimic-fn), https://github.com/sindresorhus/mimic-response.git (mimic-response), https://github.com/sindresorhus/normalize-url.git (normalize-url), https://github.com/sindresorhus/onetime.git (onetime), https://github.com/sindresorhus/open.git (open), https://github.com/sindresorhus/open-editor.git (open-editor), https://github.com/sindresorhus/opn.git (opn), https://github.com/sindresorhus/os-locale.git (os-locale), https://github.com/sindresorhus/p-cancelable.git (p-cancelable), https://github.com/sindresorhus/p-is-promise.git (p-is-promise), https://github.com/sindresorhus/p-limit.git (p-limit), https://github.com/sindresorhus/p-locate.git (p-locate), https://github.com/sindresorhus/p-map.git (p-map), https://github.com/sindresorhus/p-reduce.git (p-reduce), https://github.com/sindresorhus/p-retry.git (p-retry), https://github.com/sindresorhus/p-try.git (p-try), https://github.com/sindresorhus/package-json.git (package-json), https://github.com/sindresorhus/parent-module.git (parent-module), https://github.com/sindresorhus/parse-json.git (parse-json), https://github.com/sindresorhus/parse-ms.git (parse-ms), https://github.com/sindresorhus/path-exists.git (path-exists), https://github.com/sindresorhus/path-key.git (path-key), https://github.com/sindresorhus/path-type.git (path-type), https://github.com/sindresorhus/pify.git (pify), https://github.com/sindresorhus/pkg-dir.git (pkg-dir), https://github.com/sindresorhus/pkg-up.git (pkg-up), https://github.com/sindresorhus/plur.git (plur), https://github.com/sindresorhus/prepend-http.git (prepend-http), https://github.com/sindresorhus/pretty-bytes.git (pretty-bytes), https://github.com/sindresorhus/pretty-ms.git (pretty-ms), https://github.com/sindresorhus/proto-props.git (proto-props), https://github.com/sindresorhus/pupa.git (pupa), https://github.com/sindresorhus/quick-lru.git (quick-lru), https://github.com/sindresorhus/read-pkg.git (read-pkg), https://github.com/sindresorhus/read-pkg-up.git (read-pkg-up), https://github.com/sindresorhus/registry-url.git (registry-url), https://github.com/sindresorhus/resolve-cwd.git (resolve-cwd), https://github.com/sindresorhus/resolve-from.git (resolve-from), https://github.com/sindresorhus/restore-cursor.git (restore-cursor), https://github.com/sindresorhus/semver-diff.git (semver-diff), https://github.com/sindresorhus/shebang-regex.git (shebang-regex), https://github.com/sindresorhus/slash.git (slash), https://github.com/sindresorhus/string-length.git (string-length), https://github.com/sindresorhus/string-width.git (string-width), https://github.com/chalk/strip-ansi.git (strip-ansi), https://github.com/sindresorhus/strip-json-comments.git (strip-json-comments), https://github.com/chalk/supports-color.git (supports-color), https://github.com/sindresorhus/term-size.git (term-size), https://github.com/sindresorhus/to-readable-stream.git (to-readable-stream), https://github.com/sindresorhus/type-fest.git (type-fest), https://github.com/sindresorhus/unique-string.git (unique-string), https://github.com/sindresorhus/url-parse-lax.git (url-parse-lax), https://github.com/sindresorhus/widest-line.git (widest-line), https://github.com/chalk/wrap-ansi.git (wrap-ansi), https://github.com/sindresorhus/xdg-basedir.git (xdg-basedir). This software contains the following license and notice below:
+The following software may be included in this product: @plotly/d3-sankey. A copy of the source code may be downloaded from https://github.com/d3/d3-sankey.git. This software contains the following license and notice below:
+
+Copyright 2015, Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: @plotly/d3-sankey-circular. A copy of the source code may be downloaded from git+https://github.com/plotly/d3-sankey-circular.git. This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2017 Tom Shanley
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: @sindresorhus/is, aggregate-error, ansi-escapes, ansi-regex, ansi-styles, arrify, binary-extensions, boxen, callsites, camelcase, chalk, clean-stack, cli-boxes, cli-cursor, crypto-random-string, del, dot-prop, env-editor, escape-goat, escape-string-regexp, eslint-config-xo, eslint-config-xo-react, eslint-config-xo-typescript, eslint-formatter-pretty, eslint-plugin-ava, execa, find-up, get-stdin, get-stream, global-dirs, globals, globby, got, gzip-size, has-flag, has-yarn, import-fresh, import-local, import-modules, indent-string, internal-ip, invert-kv, irregular-plurals, is-absolute-url, is-blob, is-docker, is-finite, is-fullwidth-code-point, is-generator-fn, is-installed-globally, is-npm, is-obj, is-path-cwd, is-path-in-cwd, is-path-inside, is-root, is-svg, is-wsl, latest-version, lcid, leven, line-column-path, locate-path, log-symbols, lowercase-keys, make-dir, mem, meow, mimic-fn, mimic-response, normalize-url, onetime, open, open-editor, opn, os-locale, p-cancelable, p-is-promise, p-limit, p-locate, p-map, p-reduce, p-retry, p-try, package-json, parent-module, parse-json, parse-ms, path-exists, path-key, path-type, pify, pkg-dir, pkg-up, plur, prepend-http, pretty-bytes, pretty-ms, proto-props, pupa, quick-lru, read-pkg, read-pkg-up, registry-url, resolve-cwd, resolve-from, restore-cursor, semver-diff, shebang-regex, slash, string-length, string-width, strip-ansi, strip-json-comments, supports-color, term-size, to-readable-stream, type-fest, unique-string, url-parse-lax, widest-line, wrap-ansi, xdg-basedir. A copy of the source code may be downloaded from https://github.com/sindresorhus/is.git (@sindresorhus/is), https://github.com/sindresorhus/aggregate-error.git (aggregate-error), https://github.com/sindresorhus/ansi-escapes.git (ansi-escapes), https://github.com/chalk/ansi-regex.git (ansi-regex), https://github.com/chalk/ansi-styles.git (ansi-styles), https://github.com/sindresorhus/arrify.git (arrify), https://github.com/sindresorhus/binary-extensions.git (binary-extensions), https://github.com/sindresorhus/boxen.git (boxen), https://github.com/sindresorhus/callsites.git (callsites), https://github.com/sindresorhus/camelcase.git (camelcase), https://github.com/chalk/chalk.git (chalk), https://github.com/sindresorhus/clean-stack.git (clean-stack), https://github.com/sindresorhus/cli-boxes.git (cli-boxes), https://github.com/sindresorhus/cli-cursor.git (cli-cursor), https://github.com/sindresorhus/crypto-random-string.git (crypto-random-string), https://github.com/sindresorhus/del.git (del), https://github.com/sindresorhus/dot-prop.git (dot-prop), https://github.com/sindresorhus/env-editor.git (env-editor), https://github.com/sindresorhus/escape-goat.git (escape-goat), https://github.com/sindresorhus/escape-string-regexp.git (escape-string-regexp), https://github.com/xojs/eslint-config-xo.git (eslint-config-xo), https://github.com/xojs/eslint-config-xo-react.git (eslint-config-xo-react), https://github.com/xojs/eslint-config-xo-typescript.git (eslint-config-xo-typescript), https://github.com/sindresorhus/eslint-formatter-pretty.git (eslint-formatter-pretty), https://github.com/avajs/eslint-plugin-ava.git (eslint-plugin-ava), https://github.com/sindresorhus/execa.git (execa), https://github.com/sindresorhus/find-up.git (find-up), https://github.com/sindresorhus/get-stdin.git (get-stdin), https://github.com/sindresorhus/get-stream.git (get-stream), https://github.com/sindresorhus/global-dirs.git (global-dirs), https://github.com/sindresorhus/globals.git (globals), https://github.com/sindresorhus/globby.git (globby), https://github.com/sindresorhus/got.git (got), https://github.com/sindresorhus/gzip-size.git (gzip-size), https://github.com/sindresorhus/has-flag.git (has-flag), https://github.com/sindresorhus/has-yarn.git (has-yarn), https://github.com/sindresorhus/import-fresh.git (import-fresh), https://github.com/sindresorhus/import-local.git (import-local), https://github.com/sindresorhus/import-files.git (import-modules), https://github.com/sindresorhus/indent-string.git (indent-string), https://github.com/sindresorhus/internal-ip.git (internal-ip), https://github.com/sindresorhus/invert-kv.git (invert-kv), https://github.com/sindresorhus/irregular-plurals.git (irregular-plurals), https://github.com/sindresorhus/is-absolute-url.git (is-absolute-url), https://github.com/sindresorhus/is-blob.git (is-blob), https://github.com/sindresorhus/is-docker.git (is-docker), https://github.com/sindresorhus/is-finite.git (is-finite), https://github.com/sindresorhus/is-fullwidth-code-point.git (is-fullwidth-code-point), https://github.com/sindresorhus/is-generator-fn.git (is-generator-fn), https://github.com/sindresorhus/is-installed-globally.git (is-installed-globally), https://github.com/sindresorhus/is-npm.git (is-npm), https://github.com/sindresorhus/is-obj.git (is-obj), https://github.com/sindresorhus/is-path-cwd.git (is-path-cwd), https://github.com/sindresorhus/is-path-in-cwd.git (is-path-in-cwd), https://github.com/sindresorhus/is-path-inside.git (is-path-inside), https://github.com/sindresorhus/is-root.git (is-root), https://github.com/sindresorhus/is-svg.git (is-svg), https://github.com/sindresorhus/is-wsl.git (is-wsl), https://github.com/sindresorhus/latest-version.git (latest-version), https://github.com/sindresorhus/lcid.git (lcid), https://github.com/sindresorhus/leven.git (leven), https://github.com/sindresorhus/line-column-path.git (line-column-path), https://github.com/sindresorhus/locate-path.git (locate-path), https://github.com/sindresorhus/log-symbols.git (log-symbols), https://github.com/sindresorhus/lowercase-keys.git (lowercase-keys), https://github.com/sindresorhus/make-dir.git (make-dir), https://github.com/sindresorhus/mem.git (mem), https://github.com/sindresorhus/meow.git (meow), https://github.com/sindresorhus/mimic-fn.git (mimic-fn), https://github.com/sindresorhus/mimic-response.git (mimic-response), https://github.com/sindresorhus/normalize-url.git (normalize-url), https://github.com/sindresorhus/onetime.git (onetime), https://github.com/sindresorhus/open.git (open), https://github.com/sindresorhus/open-editor.git (open-editor), https://github.com/sindresorhus/opn.git (opn), https://github.com/sindresorhus/os-locale.git (os-locale), https://github.com/sindresorhus/p-cancelable.git (p-cancelable), https://github.com/sindresorhus/p-is-promise.git (p-is-promise), https://github.com/sindresorhus/p-limit.git (p-limit), https://github.com/sindresorhus/p-locate.git (p-locate), https://github.com/sindresorhus/p-map.git (p-map), https://github.com/sindresorhus/p-reduce.git (p-reduce), https://github.com/sindresorhus/p-retry.git (p-retry), https://github.com/sindresorhus/p-try.git (p-try), https://github.com/sindresorhus/package-json.git (package-json), https://github.com/sindresorhus/parent-module.git (parent-module), https://github.com/sindresorhus/parse-json.git (parse-json), https://github.com/sindresorhus/parse-ms.git (parse-ms), https://github.com/sindresorhus/path-exists.git (path-exists), https://github.com/sindresorhus/path-key.git (path-key), https://github.com/sindresorhus/path-type.git (path-type), https://github.com/sindresorhus/pify.git (pify), https://github.com/sindresorhus/pkg-dir.git (pkg-dir), https://github.com/sindresorhus/pkg-up.git (pkg-up), https://github.com/sindresorhus/plur.git (plur), https://github.com/sindresorhus/prepend-http.git (prepend-http), https://github.com/sindresorhus/pretty-bytes.git (pretty-bytes), https://github.com/sindresorhus/pretty-ms.git (pretty-ms), https://github.com/sindresorhus/proto-props.git (proto-props), https://github.com/sindresorhus/pupa.git (pupa), https://github.com/sindresorhus/quick-lru.git (quick-lru), https://github.com/sindresorhus/read-pkg.git (read-pkg), https://github.com/sindresorhus/read-pkg-up.git (read-pkg-up), https://github.com/sindresorhus/registry-url.git (registry-url), https://github.com/sindresorhus/resolve-cwd.git (resolve-cwd), https://github.com/sindresorhus/resolve-from.git (resolve-from), https://github.com/sindresorhus/restore-cursor.git (restore-cursor), https://github.com/sindresorhus/semver-diff.git (semver-diff), https://github.com/sindresorhus/shebang-regex.git (shebang-regex), https://github.com/sindresorhus/slash.git (slash), https://github.com/sindresorhus/string-length.git (string-length), https://github.com/sindresorhus/string-width.git (string-width), https://github.com/chalk/strip-ansi.git (strip-ansi), https://github.com/sindresorhus/strip-json-comments.git (strip-json-comments), https://github.com/chalk/supports-color.git (supports-color), https://github.com/sindresorhus/term-size.git (term-size), https://github.com/sindresorhus/to-readable-stream.git (to-readable-stream), https://github.com/sindresorhus/type-fest.git (type-fest), https://github.com/sindresorhus/unique-string.git (unique-string), https://github.com/sindresorhus/url-parse-lax.git (url-parse-lax), https://github.com/sindresorhus/widest-line.git (widest-line), https://github.com/chalk/wrap-ansi.git (wrap-ansi), https://github.com/sindresorhus/xdg-basedir.git (xdg-basedir). This software contains the following license and notice below:
 
 MIT License
 
@@ -470,7 +715,32 @@
 
 -----
 
-The following software may be included in this product: @types/babel__core, @types/babel__generator, @types/babel__template, @types/babel__traverse, @types/classnames, @types/color-name, @types/eslint-visitor-keys, @types/events, @types/glob, @types/history, @types/istanbul-lib-coverage, @types/istanbul-lib-report, @types/istanbul-reports, @types/jest, @types/jest-diff, @types/json-schema, @types/minimatch, @types/node, @types/normalize-package-data, @types/parse-json, @types/prop-types, @types/q, @types/react, @types/react-dom, @types/react-router, @types/react-router-dom, @types/react-slick, @types/stack-utils, @types/yargs, @types/yargs-parser. A copy of the source code may be downloaded from https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__core), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__generator), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__template), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__traverse), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/classnames), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/color-name), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/eslint-visitor-keys), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/events), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/glob), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/history), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-lib-coverage), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-lib-report), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-reports), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/jest), https://github.com/facebook/jest/tree/master/packages/jest-diff (@types/jest-diff), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/json-schema), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/minimatch), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/node), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/normalize-package-data), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/parse-json), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/prop-types), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/q), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-dom), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-router), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-router-dom), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-slick), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/stack-utils), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/yargs), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/yargs-parser). This software contains the following license and notice below:
+The following software may be included in this product: @turf/area, @turf/bbox, @turf/centroid, @turf/helpers, @turf/meta. A copy of the source code may be downloaded from git://github.com/Turfjs/turf.git (@turf/area), git://github.com/Turfjs/turf.git (@turf/bbox), git://github.com/Turfjs/turf.git (@turf/centroid), git://github.com/Turfjs/turf.git (@turf/helpers), git://github.com/Turfjs/turf.git (@turf/meta). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2017 TurfJS
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: @types/babel__core, @types/babel__generator, @types/babel__template, @types/babel__traverse, @types/classnames, @types/color-name, @types/d3, @types/eslint-visitor-keys, @types/events, @types/glob, @types/history, @types/istanbul-lib-coverage, @types/istanbul-lib-report, @types/istanbul-reports, @types/jest, @types/jest-diff, @types/json-schema, @types/minimatch, @types/node, @types/normalize-package-data, @types/parse-json, @types/prop-types, @types/q, @types/react, @types/react-dom, @types/react-plotly.js, @types/react-router, @types/react-router-dom, @types/react-slick, @types/stack-utils, @types/yargs, @types/yargs-parser. A copy of the source code may be downloaded from https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__core), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__generator), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__template), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/babel__traverse), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/classnames), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/color-name), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/d3), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/eslint-visitor-keys), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/events), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/glob), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/history), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-lib-coverage), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-lib-report), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/istanbul-reports), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/jest), https://github.com/facebook/jest/tree/master/packages/jest-diff (@types/jest-diff), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/json-schema), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/minimatch), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/node), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/normalize-package-data), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/parse-json), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/prop-types), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/q), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-dom), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-plotly.js), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-router), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-router-dom), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-slick), https://www.github.com/DefinitelyTyped/DefinitelyTyped.git (@types/stack-utils), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/yargs), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/yargs-parser). This software contains the following license and notice below:
 
 MIT License
 
@@ -496,7 +766,7 @@
 
 -----
 
-The following software may be included in this product: @types/node. A copy of the source code may be downloaded from https://github.com/DefinitelyTyped/DefinitelyTyped.git. This software contains the following license and notice below:
+The following software may be included in this product: @types/node, @types/plotly.js, @types/react-dom, @types/react-select, @types/react-transition-group. A copy of the source code may be downloaded from https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/node), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/plotly.js), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-dom), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-select), https://github.com/DefinitelyTyped/DefinitelyTyped.git (@types/react-transition-group). This software contains the following license and notice below:
 
 MIT License
 
@@ -1096,6 +1366,58 @@
 
 -----
 
+The following software may be included in this product: 3d-view, alpha-complex, alpha-shape, big-rat, boundary-cells, cdt2d, cell-orientation, circumradius, clean-pslg, compare-cell, compare-oriented-cell, filtered-vector, matrix-camera-controller, mouse-change, mouse-event, mouse-wheel, orbit-camera-controller, reduce-simplicial-complex, simplicial-complex-boundary, superscript-text, text-cache, to-px, turntable-camera-controller. A copy of the source code may be downloaded from https://github.com/mikolalysenko/3d-view.git (3d-view), https://github.com/mikolalysenko/alpha-complex.git (alpha-complex), https://github.com/mikolalysenko/alpha-shape.git (alpha-shape), https://github.com/rat-nest/big-rat.git (big-rat), git://github.com/mikolalysenko/boundary-cells.git (boundary-cells), https://github.com/mikolalysenko/cdt2d.git (cdt2d), https://github.com/mikolalysenko/cell-orientation.git (cell-orientation), https://github.com/mikolalysenko/circumradius.git (circumradius), https://github.com/mikolalysenko/clean-pslg.git (clean-pslg), https://github.com/mikolalysenko/compare-cells.git (compare-cell), https://github.com/mikolalysenko/compare-oriented-cell.git (compare-oriented-cell), https://github.com/mikolalysenko/filtered-vector.git (filtered-vector), https://github.com/mikolalysenko/matrix-camera-controller.git (matrix-camera-controller), https://github.com/mikolalysenko/mouse-change.git (mouse-change), https://github.com/mikolalysenko/mouse-event.git (mouse-event), https://github.com/mikolalysenko/mouse-wheel.git (mouse-wheel), https://github.com/mikolalysenko/orbit-camera-controller.git (orbit-camera-controller), https://github.com/mikolalysenko/reduce-simplicial-complex.git (reduce-simplicial-complex), https://github.com/mikolalysenko/simplicial-complex-boundary.git (simplicial-complex-boundary), https://github.com/mikolalysenko/superscript-text.git (superscript-text), https://github.com/gl-vis/text-cache.git (text-cache), https://github.com/mikolalysenko/to-px.git (to-px), https://github.com/mikolalysenko/turntable-camera-controller.git (turntable-camera-controller). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: a-big-triangle, affine-hull, almost-equal, barycentric, binary-search-bounds, bit-twiddle, box-intersect, circumcenter, compare-angle, cubic-hermite, cwise, cwise-compiler, cwise-parser, delaunay-triangulate, double-bits, dup, edges-to-adjacency-list, extract-frustum-planes, functional-red-black-tree, gl-buffer, gl-contour2d, gl-error3d, gl-fbo, gl-heatmap2d, gl-line3d, gl-matrix-invert, gl-mesh3d, gl-plot2d, gl-pointcloud2d, gl-scatter3d, gl-select-box, gl-select-static, gl-shader, gl-spikes2d, gl-spikes3d, gl-state, gl-texture2d, gl-vao, incremental-convex-hull, interval-tree-1d, invert-permutation, iota-array, marching-simplex-table, monotone-convex-hull-2d, ndarray, ndarray-extract-contour, ndarray-fill, ndarray-gradient, ndarray-homography, ndarray-linear-interpolate, ndarray-ops, ndarray-pack, ndarray-scratch, ndarray-sort, ndarray-warp, nextafter, normals, permutation-parity, permutation-rank, planar-dual, planar-graph-to-polyline, point-in-big-polygon, rat-vec, robust-compress, robust-determinant, robust-dot-product, robust-linear-solve, robust-orientation, robust-product, robust-scale, robust-segment-intersect, robust-subtract, robust-sum, signum, simplicial-complex, simplicial-complex-contour, simplify-planar-graph, slab-decomposition, split-polygon, strongly-connected-components, surface-nets, triangulate-hypercube, triangulate-polyline, two-product, two-sum, typedarray-pool, union-find, uniq, vectorize-text, zero-crossings. A copy of the source code may be downloaded from git://github.com/mikolalysenko/a-big-triangle.git (a-big-triangle), https://github.com/mikolalysenko/affine-hull.git (affine-hull), git://github.com/mikolalysenko/almost-equal.git (almost-equal), git://github.com/mikolalysenko/barycentric.git (barycentric), git://github.com/mikolalysenko/binary-search-bounds.git (binary-search-bounds), git://github.com/mikolalysenko/bit-twiddle.git (bit-twiddle), https://github.com/mikolalysenko/box-intersect.git (box-intersect), git://github.com/mikolalysenko/circumcenter.git (circumcenter), git://github.com/mikolalysenko/compare-angle.git (compare-angle), git://github.com/mikolalysenko/cubic-hermite.git (cubic-hermite), git://github.com/scijs/cwise.git (cwise), git://github.com/scijs/cwise-compiler.git (cwise-compiler), git://github.com/scijs/cwise-parser.git (cwise-parser), git://github.com/mikolalysenko/delaunay-triangulate.git (delaunay-triangulate), git://github.com/mikolalysenko/double-bits.git (double-bits), git://github.com/mikolalysenko/dup.git (dup), git://github.com/mikolalysenko/edges-to-adjacency-list.git (edges-to-adjacency-list), git://github.com/mikolalysenko/extract-frustum-planes.git (extract-frustum-planes), git://github.com/mikolalysenko/functional-red-black-tree.git (functional-red-black-tree), git://github.com/stackgl/gl-buffer.git (gl-buffer), git+https://github.com/gl-vis/gl-contour2d.git (gl-contour2d), git://github.com/gl-vis/gl-error3d.git (gl-error3d), git://github.com/stackgl/gl-fbo.git (gl-fbo), git+https://github.com/gl-vis/gl-heatmap2d.git (gl-heatmap2d), git://github.com/gl-vis/gl-line3d.git (gl-line3d), git://github.com/mikolalysenko/gl-matrix-invert.git (gl-matrix-invert), git://github.com/gl-vis/gl-mesh3d.git (gl-mesh3d), git+https://github.com/gl-vis/gl-plot2d.git (gl-plot2d), git+https://github.com/gl-vis/gl-pointcloud2d.git (gl-pointcloud2d), git://github.com/gl-vis/gl-scatter3d.git (gl-scatter3d), git+https://github.com/gl-vis/gl-select-box.git (gl-select-box), git://github.com/gl-vis/gl-select-static.git (gl-select-static), git://github.com/stackgl/gl-shader.git (gl-shader), git+https://github.com/gl-vis/gl-spikes2d.git (gl-spikes2d), git://github.com/gl-vis/gl-spikes3d.git (gl-spikes3d), git://github.com/mikolalysenko/gl-state.git (gl-state), git://github.com/stackgl/gl-texture2d.git (gl-texture2d), git://github.com/stackgl/gl-vao.git (gl-vao), git://github.com/mikolalysenko/incremental-convex-hull.git (incremental-convex-hull), git://github.com/mikolalysenko/interval-tree-1d.git (interval-tree-1d), git://github.com/mikolalysenko/invert-permutation.git (invert-permutation), git://github.com/mikolalysenko/iota-array.git (iota-array), https://github.com/mikolalysenko/marching-simplex-table.git (marching-simplex-table), git://github.com/mikolalysenko/monotone-convex-hull-2d.git (monotone-convex-hull-2d), git://github.com/mikolalysenko/ndarray.git (ndarray), git://github.com/mikolalysenko/ndarray-extract-contour.git (ndarray-extract-contour), git://github.com/mikolalysenko/ndarray-fill.git (ndarray-fill), git://github.com/mikolalysenko/ndarray-gradient.git (ndarray-gradient), git://github.com/mikolalysenko/ndarray-homography.git (ndarray-homography), git://github.com/mikolalysenko/ndarray-linear-interpolate.git (ndarray-linear-interpolate), git://github.com/mikolalysenko/ndarray-ops.git (ndarray-ops), git://github.com/mikolalysenko/ndarray-pack.git (ndarray-pack), git://github.com/mikolalysenko/ndarray-scratch.git (ndarray-scratch), git://github.com/mikolalysenko/ndarray-sort.git (ndarray-sort), git://github.com/mikolalysenko/ndarray-warp.git (ndarray-warp), git://github.com/mikolalysenko/nextafter.git (nextafter), git://github.com/mikolalysenko/normals.git (normals), git://github.com/mikolalysenko/permutation-parity.git (permutation-parity), git://github.com/mikolalysenko/permutation-rank.git (permutation-rank), git://github.com/mikolalysenko/planar-dual.git (planar-dual), git://github.com/mikolalysenko/planar-graph-to-polyline.git (planar-graph-to-polyline), git://github.com/mikolalysenko/point-in-big-polygon.git (point-in-big-polygon), git+https://github.com/rat-nest/rat-vec.git (rat-vec), git://github.com/mikolalysenko/robust-compress.git (robust-compress), git://github.com/mikolalysenko/robust-determinant.git (robust-determinant), git://github.com/mikolalysenko/robust-dot-product.git (robust-dot-product), git://github.com/mikolalysenko/robust-linear-solve.git (robust-linear-solve), git://github.com/mikolalysenko/robust-orientation.git (robust-orientation), git://github.com/mikolalysenko/robust-product.git (robust-product), git://github.com/mikolalysenko/robust-scale.git (robust-scale), git://github.com/mikolalysenko/robust-segment-intersect.git (robust-segment-intersect), git://github.com/mikolalysenko/robust-subtract.git (robust-subtract), git://github.com/mikolalysenko/robust-sum.git (robust-sum), git://github.com/mikolalysenko/signum.git (signum), git://github.com/mikolalysenko/simplicial-complex.git (simplicial-complex), https://github.com/mikolalysenko/simplicial-complex-contour.git (simplicial-complex-contour), git://github.com/mikolalysenko/simplify-planar-graph.git (simplify-planar-graph), git://github.com/mikolalysenko/slab-decomposition.git (slab-decomposition), git://github.com/mikolalysenko/split-polygon.git (split-polygon), git://github.com/mikolalysenko/strongly-connected-components.git (strongly-connected-components), git://github.com/mikolalysenko/surface-nets.git (surface-nets), git://github.com/mikolalysenko/triangulate-hypercube.git (triangulate-hypercube), git://github.com/mikolalysenko/triangulate-polyline.git (triangulate-polyline), git://github.com/mikolalysenko/two-product.git (two-product), git://github.com/mikolalysenko/two-sum.git (two-sum), git://github.com/mikolalysenko/typedarray-pool.git (typedarray-pool), git://github.com/mikolalysenko/union-find.git (union-find), git://github.com/mikolalysenko/uniq.git (uniq), git://github.com/mikolalysenko/vectorize-text.git (vectorize-text), git://github.com/mikolalysenko/zero-crossings.git (zero-crossings). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2013 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: abab. A copy of the source code may be downloaded from git+https://github.com/jsdom/abab.git. This software contains the following license and notice below:
 
 Copyright © 2019 W3C and Jeff Carpenter \<jeffcarp@chromium.org\>
@@ -1166,6 +1488,32 @@
 
 -----
 
+The following software may be included in this product: acorn-dynamic-import. A copy of the source code may be downloaded from https://github.com/kesne/acorn-dynamic-import. This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2016 Jordan Gensler
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: acorn-globals, is-promise. A copy of the source code may be downloaded from https://github.com/ForbesLindesay/acorn-globals.git (acorn-globals), https://github.com/then/is-promise.git (is-promise). This software contains the following license and notice below:
 
 Copyright (c) 2014 Forbes Lindesay
@@ -1228,6 +1576,31 @@
 
 -----
 
+The following software may be included in this product: add-line-numbers, get-canvas-context, glsl-token-inject-block. A copy of the source code may be downloaded from git://github.com/Jam3/add-line-numbers.git (add-line-numbers), git://github.com/Jam3/get-canvas-context.git (get-canvas-context), git://github.com/Jam3/glsl-token-inject-block.git (glsl-token-inject-block). This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright (c) 2015 Jam3
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: address. A copy of the source code may be downloaded from git://github.com/node-modules/address.git. This software contains the following license and notice below:
 
 This software is licensed under the MIT License.
@@ -1333,6 +1706,32 @@
 
 -----
 
+The following software may be included in this product: align-text, assign-symbols, center-align, contains-path, define-property, is-accessor-descriptor, is-data-descriptor, is-extendable, lazy-cache, pascalcase, right-align, shallow-clone, unc-path-regex. A copy of the source code may be downloaded from git://github.com/jonschlinkert/align-text.git (align-text), https://github.com/jonschlinkert/assign-symbols.git (assign-symbols), https://github.com/jonschlinkert/center-align.git (center-align), https://github.com/jonschlinkert/contains-path.git (contains-path), https://github.com/jonschlinkert/define-property.git (define-property), https://github.com/jonschlinkert/is-accessor-descriptor.git (is-accessor-descriptor), https://github.com/jonschlinkert/is-data-descriptor.git (is-data-descriptor), https://github.com/jonschlinkert/is-extendable.git (is-extendable), https://github.com/jonschlinkert/lazy-cache.git (lazy-cache), https://github.com/jonschlinkert/pascalcase.git (pascalcase), git://github.com/jonschlinkert/right-align.git (right-align), https://github.com/jonschlinkert/shallow-clone.git (shallow-clone), https://github.com/regexhq/unc-path-regex.git (unc-path-regex). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015, Jon Schlinkert.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: alphanum-sort, postcss-minify-font-values, postcss-value-parser. A copy of the source code may be downloaded from https://github.com/TrySound/alphanum-sort.git (alphanum-sort), https://github.com/cssnano/cssnano.git (postcss-minify-font-values), https://github.com/TrySound/postcss-value-parser.git (postcss-value-parser). This software contains the following license and notice below:
 
 Copyright (c) Bogdan Chadkin <trysound@yandex.ru>
@@ -1360,6 +1759,69 @@
 
 -----
 
+The following software may be included in this product: amdefine. A copy of the source code may be downloaded from https://github.com/jrburke/amdefine.git. This software contains the following license and notice below:
+
+amdefine is released under two licenses: new BSD, and MIT. You may pick the
+license that best suits your development needs. The text of both licenses are
+provided below.
+
+
+The "New" BSD License:
+----------------------
+
+Copyright (c) 2011-2016, The Dojo Foundation
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+  * Redistributions of source code must retain the above copyright notice, this
+    list of conditions and the following disclaimer.
+  * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+  * Neither the name of the Dojo Foundation nor the names of its contributors
+    may be used to endorse or promote products derived from this software
+    without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+
+MIT License
+-----------
+
+Copyright (c) 2011-2016, The Dojo Foundation
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: ansi-align, which-module. A copy of the source code may be downloaded from git+https://github.com/nexdrew/ansi-align.git (ansi-align), git+https://github.com/nexdrew/which-module.git (which-module). This software contains the following license and notice below:
 
 Copyright (c) 2016, Contributors
@@ -1624,7 +2086,7 @@
 
 -----
 
-The following software may be included in this product: ansi-regex, ansi-styles, array-find-index, array-union, array-uniq, arrify, buf-compare, caller-callsite, caller-path, callsites, camelcase, camelcase-keys, chalk, code-point-at, core-assert, crypto-random-string, decamelize, deep-strict-equal, detect-newline, dot-prop, escape-string-regexp, find-up, get-stream, globby, has-ansi, import-cwd, import-fresh, import-from, import-lazy, ip-regex, is-absolute-url, is-binary-path, is-fullwidth-code-point, is-obj, is-path-inside, is-plain-obj, is-stream, is-wsl, js-types, load-json-file, locate-path, loud-rejection, lowercase-keys, map-obj, normalize-url, npm-run-path, number-is-nan, object-assign, os-tmpdir, p-defer, p-each-series, p-finally, p-locate, p-reduce, p-try, parse-json, path-exists, path-is-absolute, path-key, path-type, pify, pkg-dir, pkg-up, prepend-http, query-string, read-pkg, read-pkg-up, redent, resolve-cwd, resolve-from, semver-diff, shebang-regex, sort-keys, string-width, strip-ansi, strip-bom, strip-eof, strip-indent, strip-json-comments, supports-color, term-size, trim-newlines, unique-string, wrap-ansi, xdg-basedir. A copy of the source code may be downloaded from https://github.com/chalk/ansi-regex.git (ansi-regex), https://github.com/chalk/ansi-styles.git (ansi-styles), https://github.com/sindresorhus/array-find-index.git (array-find-index), https://github.com/sindresorhus/array-union.git (array-union), https://github.com/sindresorhus/array-uniq.git (array-uniq), https://github.com/sindresorhus/arrify.git (arrify), https://github.com/sindresorhus/buf-compare.git (buf-compare), https://github.com/sindresorhus/caller-callsite.git (caller-callsite), https://github.com/sindresorhus/caller-path.git (caller-path), https://github.com/sindresorhus/callsites.git (callsites), https://github.com/sindresorhus/camelcase.git (camelcase), https://github.com/sindresorhus/camelcase-keys.git (camelcase-keys), https://github.com/chalk/chalk.git (chalk), https://github.com/sindresorhus/code-point-at.git (code-point-at), https://github.com/sindresorhus/core-assert.git (core-assert), https://github.com/sindresorhus/crypto-random-string.git (crypto-random-string), https://github.com/sindresorhus/decamelize.git (decamelize), https://github.com/sindresorhus/deep-strict-equal.git (deep-strict-equal), https://github.com/sindresorhus/detect-newline.git (detect-newline), https://github.com/sindresorhus/dot-prop.git (dot-prop), https://github.com/sindresorhus/escape-string-regexp.git (escape-string-regexp), https://github.com/sindresorhus/find-up.git (find-up), https://github.com/sindresorhus/get-stream.git (get-stream), https://github.com/sindresorhus/globby.git (globby), https://github.com/sindresorhus/has-ansi.git (has-ansi), https://github.com/sindresorhus/import-cwd.git (import-cwd), https://github.com/sindresorhus/import-fresh.git (import-fresh), https://github.com/sindresorhus/import-from.git (import-from), https://github.com/sindresorhus/import-lazy.git (import-lazy), https://github.com/sindresorhus/ip-regex.git (ip-regex), https://github.com/sindresorhus/is-absolute-url.git (is-absolute-url), https://github.com/sindresorhus/is-binary-path.git (is-binary-path), https://github.com/sindresorhus/is-fullwidth-code-point.git (is-fullwidth-code-point), https://github.com/sindresorhus/is-obj.git (is-obj), https://github.com/sindresorhus/is-path-inside.git (is-path-inside), https://github.com/sindresorhus/is-plain-obj.git (is-plain-obj), https://github.com/sindresorhus/is-stream.git (is-stream), https://github.com/sindresorhus/is-wsl.git (is-wsl), https://github.com/sindresorhus/js-types.git (js-types), https://github.com/sindresorhus/load-json-file.git (load-json-file), https://github.com/sindresorhus/locate-path.git (locate-path), https://github.com/sindresorhus/loud-rejection.git (loud-rejection), https://github.com/sindresorhus/lowercase-keys.git (lowercase-keys), https://github.com/sindresorhus/map-obj.git (map-obj), https://github.com/sindresorhus/normalize-url.git (normalize-url), https://github.com/sindresorhus/npm-run-path.git (npm-run-path), https://github.com/sindresorhus/number-is-nan.git (number-is-nan), https://github.com/sindresorhus/object-assign.git (object-assign), https://github.com/sindresorhus/os-tmpdir.git (os-tmpdir), https://github.com/sindresorhus/p-defer.git (p-defer), https://github.com/sindresorhus/p-each-series.git (p-each-series), https://github.com/sindresorhus/p-finally.git (p-finally), https://github.com/sindresorhus/p-locate.git (p-locate), https://github.com/sindresorhus/p-reduce.git (p-reduce), https://github.com/sindresorhus/p-try.git (p-try), https://github.com/sindresorhus/parse-json.git (parse-json), https://github.com/sindresorhus/path-exists.git (path-exists), https://github.com/sindresorhus/path-is-absolute.git (path-is-absolute), https://github.com/sindresorhus/path-key.git (path-key), https://github.com/sindresorhus/path-type.git (path-type), https://github.com/sindresorhus/pify.git (pify), https://github.com/sindresorhus/pkg-dir.git (pkg-dir), https://github.com/sindresorhus/pkg-up.git (pkg-up), https://github.com/sindresorhus/prepend-http.git (prepend-http), https://github.com/sindresorhus/query-string.git (query-string), https://github.com/sindresorhus/read-pkg.git (read-pkg), https://github.com/sindresorhus/read-pkg-up.git (read-pkg-up), https://github.com/sindresorhus/redent.git (redent), https://github.com/sindresorhus/resolve-cwd.git (resolve-cwd), https://github.com/sindresorhus/resolve-from.git (resolve-from), https://github.com/sindresorhus/semver-diff.git (semver-diff), https://github.com/sindresorhus/shebang-regex.git (shebang-regex), https://github.com/sindresorhus/sort-keys.git (sort-keys), https://github.com/sindresorhus/string-width.git (string-width), https://github.com/chalk/strip-ansi.git (strip-ansi), https://github.com/sindresorhus/strip-bom.git (strip-bom), https://github.com/sindresorhus/strip-eof.git (strip-eof), https://github.com/sindresorhus/strip-indent.git (strip-indent), https://github.com/sindresorhus/strip-json-comments.git (strip-json-comments), https://github.com/chalk/supports-color.git (supports-color), https://github.com/sindresorhus/term-size.git (term-size), https://github.com/sindresorhus/trim-newlines.git (trim-newlines), https://github.com/sindresorhus/unique-string.git (unique-string), https://github.com/chalk/wrap-ansi.git (wrap-ansi), https://github.com/sindresorhus/xdg-basedir.git (xdg-basedir). This software contains the following license and notice below:
+The following software may be included in this product: ansi-regex, ansi-styles, array-find-index, array-union, array-uniq, arrify, buf-compare, caller-callsite, caller-path, callsites, camelcase, camelcase-keys, chalk, code-point-at, core-assert, crypto-random-string, decamelize, deep-strict-equal, detect-newline, dot-prop, escape-string-regexp, find-up, get-stream, globby, has-ansi, import-cwd, import-fresh, import-from, import-lazy, ip-regex, is-absolute-url, is-binary-path, is-fullwidth-code-point, is-obj, is-path-inside, is-plain-obj, is-stream, is-wsl, js-types, load-json-file, locate-path, loud-rejection, lowercase-keys, map-obj, math-log2, normalize-url, npm-run-path, number-is-integer, number-is-nan, object-assign, os-homedir, os-tmpdir, p-defer, p-each-series, p-finally, p-locate, p-reduce, p-try, parse-json, path-exists, path-is-absolute, path-key, path-type, pify, pkg-dir, pkg-up, prepend-http, query-string, read-pkg, read-pkg-up, redent, resolve-cwd, resolve-from, semver-diff, shebang-regex, sort-keys, string-width, strip-ansi, strip-bom, strip-eof, strip-indent, strip-json-comments, supports-color, term-size, trim-newlines, unique-string, wrap-ansi, xdg-basedir. A copy of the source code may be downloaded from https://github.com/chalk/ansi-regex.git (ansi-regex), https://github.com/chalk/ansi-styles.git (ansi-styles), https://github.com/sindresorhus/array-find-index.git (array-find-index), https://github.com/sindresorhus/array-union.git (array-union), https://github.com/sindresorhus/array-uniq.git (array-uniq), https://github.com/sindresorhus/arrify.git (arrify), https://github.com/sindresorhus/buf-compare.git (buf-compare), https://github.com/sindresorhus/caller-callsite.git (caller-callsite), https://github.com/sindresorhus/caller-path.git (caller-path), https://github.com/sindresorhus/callsites.git (callsites), https://github.com/sindresorhus/camelcase.git (camelcase), https://github.com/sindresorhus/camelcase-keys.git (camelcase-keys), https://github.com/chalk/chalk.git (chalk), https://github.com/sindresorhus/code-point-at.git (code-point-at), https://github.com/sindresorhus/core-assert.git (core-assert), https://github.com/sindresorhus/crypto-random-string.git (crypto-random-string), https://github.com/sindresorhus/decamelize.git (decamelize), https://github.com/sindresorhus/deep-strict-equal.git (deep-strict-equal), https://github.com/sindresorhus/detect-newline.git (detect-newline), https://github.com/sindresorhus/dot-prop.git (dot-prop), https://github.com/sindresorhus/escape-string-regexp.git (escape-string-regexp), https://github.com/sindresorhus/find-up.git (find-up), https://github.com/sindresorhus/get-stream.git (get-stream), https://github.com/sindresorhus/globby.git (globby), https://github.com/sindresorhus/has-ansi.git (has-ansi), https://github.com/sindresorhus/import-cwd.git (import-cwd), https://github.com/sindresorhus/import-fresh.git (import-fresh), https://github.com/sindresorhus/import-from.git (import-from), https://github.com/sindresorhus/import-lazy.git (import-lazy), https://github.com/sindresorhus/ip-regex.git (ip-regex), https://github.com/sindresorhus/is-absolute-url.git (is-absolute-url), https://github.com/sindresorhus/is-binary-path.git (is-binary-path), https://github.com/sindresorhus/is-fullwidth-code-point.git (is-fullwidth-code-point), https://github.com/sindresorhus/is-obj.git (is-obj), https://github.com/sindresorhus/is-path-inside.git (is-path-inside), https://github.com/sindresorhus/is-plain-obj.git (is-plain-obj), https://github.com/sindresorhus/is-stream.git (is-stream), https://github.com/sindresorhus/is-wsl.git (is-wsl), https://github.com/sindresorhus/js-types.git (js-types), https://github.com/sindresorhus/load-json-file.git (load-json-file), https://github.com/sindresorhus/locate-path.git (locate-path), https://github.com/sindresorhus/loud-rejection.git (loud-rejection), https://github.com/sindresorhus/lowercase-keys.git (lowercase-keys), https://github.com/sindresorhus/map-obj.git (map-obj), https://github.com/sindresorhus/math-log2.git (math-log2), https://github.com/sindresorhus/normalize-url.git (normalize-url), https://github.com/sindresorhus/npm-run-path.git (npm-run-path), https://github.com/sindresorhus/number-is-integer.git (number-is-integer), https://github.com/sindresorhus/number-is-nan.git (number-is-nan), https://github.com/sindresorhus/object-assign.git (object-assign), https://github.com/sindresorhus/os-homedir.git (os-homedir), https://github.com/sindresorhus/os-tmpdir.git (os-tmpdir), https://github.com/sindresorhus/p-defer.git (p-defer), https://github.com/sindresorhus/p-each-series.git (p-each-series), https://github.com/sindresorhus/p-finally.git (p-finally), https://github.com/sindresorhus/p-locate.git (p-locate), https://github.com/sindresorhus/p-reduce.git (p-reduce), https://github.com/sindresorhus/p-try.git (p-try), https://github.com/sindresorhus/parse-json.git (parse-json), https://github.com/sindresorhus/path-exists.git (path-exists), https://github.com/sindresorhus/path-is-absolute.git (path-is-absolute), https://github.com/sindresorhus/path-key.git (path-key), https://github.com/sindresorhus/path-type.git (path-type), https://github.com/sindresorhus/pify.git (pify), https://github.com/sindresorhus/pkg-dir.git (pkg-dir), https://github.com/sindresorhus/pkg-up.git (pkg-up), https://github.com/sindresorhus/prepend-http.git (prepend-http), https://github.com/sindresorhus/query-string.git (query-string), https://github.com/sindresorhus/read-pkg.git (read-pkg), https://github.com/sindresorhus/read-pkg-up.git (read-pkg-up), https://github.com/sindresorhus/redent.git (redent), https://github.com/sindresorhus/resolve-cwd.git (resolve-cwd), https://github.com/sindresorhus/resolve-from.git (resolve-from), https://github.com/sindresorhus/semver-diff.git (semver-diff), https://github.com/sindresorhus/shebang-regex.git (shebang-regex), https://github.com/sindresorhus/sort-keys.git (sort-keys), https://github.com/sindresorhus/string-width.git (string-width), https://github.com/chalk/strip-ansi.git (strip-ansi), https://github.com/sindresorhus/strip-bom.git (strip-bom), https://github.com/sindresorhus/strip-eof.git (strip-eof), https://github.com/sindresorhus/strip-indent.git (strip-indent), https://github.com/sindresorhus/strip-json-comments.git (strip-json-comments), https://github.com/chalk/supports-color.git (supports-color), https://github.com/sindresorhus/term-size.git (term-size), https://github.com/sindresorhus/trim-newlines.git (trim-newlines), https://github.com/sindresorhus/unique-string.git (unique-string), https://github.com/chalk/wrap-ansi.git (wrap-ansi), https://github.com/sindresorhus/xdg-basedir.git (xdg-basedir). This software contains the following license and notice below:
 
 The MIT License (MIT)
 
@@ -2175,6 +2637,57 @@
 
 -----
 
+The following software may be included in this product: array-range, lerp, mat4-decompose, mat4-interpolate, mat4-recompose, mouse-event-offset, parse-unit, quat-slerp, webgl-context. A copy of the source code may be downloaded from git://github.com/mattdesl/array-range.git (array-range), git://github.com/mattdesl/lerp.git (lerp), git://github.com/mattdesl/mat4-decompose.git (mat4-decompose), git://github.com/mattdesl/mat4-interpolate.git (mat4-interpolate), git://github.com/mattdesl/mat4-recompose.git (mat4-recompose), git://github.com/mattdesl/mouse-event-offset.git (mouse-event-offset), git://github.com/mattdesl/parse-unit.git (parse-unit), git://github.com/mattdesl/quat-slerp.git (quat-slerp), git://github.com/mattdesl/webgl-context.git (webgl-context). This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright (c) 2014 Matt DesLauriers
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: array-rearrange. A copy of the source code may be downloaded from git+ssh://git@github.com/dfcreative/array-rearrange.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+2018 (c) Dmitry Ivanov <df.creative@gmail.com>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: array-unique, is-extglob, is-number. A copy of the source code may be downloaded from https://github.com/jonschlinkert/array-unique.git (array-unique), https://github.com/jonschlinkert/is-extglob.git (is-extglob), https://github.com/jonschlinkert/is-number.git (is-number). This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -2275,7 +2788,7 @@
 
 -----
 
-The following software may be included in this product: assert, util. A copy of the source code may be downloaded from git://github.com/browserify/commonjs-assert.git (assert), git://github.com/defunctzombie/node-util (util). This software contains the following license and notice below:
+The following software may be included in this product: assert, readable-stream, util. A copy of the source code may be downloaded from git://github.com/browserify/commonjs-assert.git (assert), git://github.com/isaacs/readable-stream (readable-stream), git://github.com/defunctzombie/node-util (util). This software contains the following license and notice below:
 
 Copyright Joyent, Inc. and other Node contributors. All rights reserved.
 Permission is hereby granted, free of charge, to any person obtaining a copy
@@ -2298,32 +2811,6 @@
 
 -----
 
-The following software may be included in this product: assign-symbols, contains-path, define-property, is-accessor-descriptor, is-data-descriptor, is-extendable, lazy-cache, pascalcase, shallow-clone, unc-path-regex. A copy of the source code may be downloaded from https://github.com/jonschlinkert/assign-symbols.git (assign-symbols), https://github.com/jonschlinkert/contains-path.git (contains-path), https://github.com/jonschlinkert/define-property.git (define-property), https://github.com/jonschlinkert/is-accessor-descriptor.git (is-accessor-descriptor), https://github.com/jonschlinkert/is-data-descriptor.git (is-data-descriptor), https://github.com/jonschlinkert/is-extendable.git (is-extendable), https://github.com/jonschlinkert/lazy-cache.git (lazy-cache), https://github.com/jonschlinkert/pascalcase.git (pascalcase), https://github.com/jonschlinkert/shallow-clone.git (shallow-clone), https://github.com/regexhq/unc-path-regex.git (unc-path-regex). This software contains the following license and notice below:
-
-The MIT License (MIT)
-
-Copyright (c) 2015, Jon Schlinkert.
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
-
------
-
 The following software may be included in this product: astral-regex, dir-glob, shebang-command. A copy of the source code may be downloaded from https://github.com/kevva/astral-regex.git (astral-regex), https://github.com/kevva/dir-glob.git (dir-glob), https://github.com/kevva/shebang-command.git (shebang-command). This software contains the following license and notice below:
 
 MIT License
@@ -2647,6 +3134,29 @@
 
 -----
 
+The following software may be included in this product: atob-lite, canvas-fit, concat-map, defined, element-size, falafel, font-atlas, gamma, is-typedarray, json-stable-stringify, json-stable-stringify-without-jsonify, minimist, object-inspect, path-browserify, quote-stream, resolve, resumer, right-now, safe-regex, semver-compare, shallow-copy, static-eval, static-module, text-table, tty-browserify, wordwrap. A copy of the source code may be downloaded from git://github.com/hughsk/atob-lite.git (atob-lite), git://github.com/hughsk/canvas-fit.git (canvas-fit), git://github.com/substack/node-concat-map.git (concat-map), git://github.com/substack/defined.git (defined), git://github.com/hughsk/element-size.git (element-size), git://github.com/substack/node-falafel.git (falafel), git://github.com/hughsk/font-atlas.git (font-atlas), http://github.com/substack/gamma.js.git (gamma), git://github.com/hughsk/is-typedarray.git (is-typedarray), git://github.com/substack/json-stable-stringify.git (json-stable-stringify), git://github.com/samn/json-stable-stringify.git (json-stable-stringify-without-jsonify), git://github.com/substack/minimist.git (minimist), git://github.com/substack/object-inspect.git (object-inspect), git://github.com/substack/path-browserify.git (path-browserify), git://github.com/substack/quote-stream.git (quote-stream), git://github.com/substack/node-resolve.git (resolve), git://github.com/substack/resumer.git (resumer), git://github.com/hughsk/right-now.git (right-now), git://github.com/substack/safe-regex.git (safe-regex), git://github.com/substack/semver-compare.git (semver-compare), git://github.com/substack/shallow-copy.git (shallow-copy), git://github.com/substack/static-eval.git (static-eval), git://github.com/substack/static-module.git (static-module), git://github.com/substack/text-table.git (text-table), git://github.com/substack/tty-browserify.git (tty-browserify), git://github.com/substack/node-wordwrap.git (wordwrap). This software contains the following license and notice below:
+
+This software is released under the MIT license:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: autoprefixer, postcss, postcss-safe-parser. A copy of the source code may be downloaded from https://github.com/postcss/autoprefixer.git (autoprefixer), https://github.com/postcss/postcss.git (postcss), https://github.com/postcss/postcss-safe-parser.git (postcss-safe-parser). This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -3154,6 +3664,32 @@
 
 -----
 
+The following software may be included in this product: binary-search-bounds. A copy of the source code may be downloaded from git://github.com/mikolalysenko/binary-search-bounds.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2013-2015 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: bindings. A copy of the source code may be downloaded from git://github.com/TooTallNate/node-bindings.git. This software contains the following license and notice below:
 
 (The MIT License)
@@ -3181,6 +3717,24 @@
 
 -----
 
+The following software may be included in this product: bl. A copy of the source code may be downloaded from https://github.com/rvagg/bl.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+=====================
+
+Copyright (c) 2013-2016 bl contributors
+----------------------------------
+
+*bl contributors listed at <https://github.com/rvagg/bl#contributors>*
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: bluebird. A copy of the source code may be downloaded from git://github.com/petkaantonov/bluebird.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -3261,7 +3815,7 @@
 
 -----
 
-The following software may be included in this product: brace-expansion. A copy of the source code may be downloaded from git://github.com/juliangruber/brace-expansion.git. This software contains the following license and notice below:
+The following software may be included in this product: brace-expansion, isarray. A copy of the source code may be downloaded from git://github.com/juliangruber/brace-expansion.git (brace-expansion), git://github.com/juliangruber/isarray.git (isarray). This software contains the following license and notice below:
 
 MIT License
 
@@ -3600,6 +4154,58 @@
 
 -----
 
+The following software may be included in this product: buble. A copy of the source code may be downloaded from git+https://github.com/bublejs/buble.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 Rich Harris and contributors
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: bubleify. A copy of the source code may be downloaded from git://github.com/garthenweb/bubleify. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 Jannick Garthen
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: buffer. A copy of the source code may be downloaded from git://github.com/feross/buffer.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -4490,6 +5096,32 @@
 
 -----
 
+The following software may be included in this product: clamp. A copy of the source code may be downloaded from git://github.com/hughsk/clamp. This software contains the following license and notice below:
+
+## The MIT License (MIT) ##
+
+Copyright (c) 2013 Hugh Kennedy
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: class-utils. A copy of the source code may be downloaded from https://github.com/jonschlinkert/class-utils.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -4640,7 +5272,7 @@
 
 -----
 
-The following software may be included in this product: clone-deep, extend-shallow, mixin-object. A copy of the source code may be downloaded from https://github.com/jonschlinkert/clone-deep.git (clone-deep), https://github.com/jonschlinkert/extend-shallow.git (extend-shallow), https://github.com/jonschlinkert/mixin-object.git (mixin-object). This software contains the following license and notice below:
+The following software may be included in this product: clone-deep, extend-shallow, longest, mixin-object, pad-left. A copy of the source code may be downloaded from https://github.com/jonschlinkert/clone-deep.git (clone-deep), https://github.com/jonschlinkert/extend-shallow.git (extend-shallow), https://github.com/jonschlinkert/longest.git (longest), https://github.com/jonschlinkert/mixin-object.git (mixin-object), git://github.com/jonschlinkert/pad-left.git (pad-left). This software contains the following license and notice below:
 
 The MIT License (MIT)
 
@@ -4808,6 +5440,61 @@
 
 -----
 
+The following software may be included in this product: color-parse. A copy of the source code may be downloaded from https://github.com/colorjs/color-parse. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Dmitry Ivanov
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: color-space, mumath. A copy of the source code may be downloaded from http://github.com/colorjs/color-space.git (color-space), git://github.com/dfcreative/mumath.git (mumath). This software contains the following license and notice below:
+
+This is free and unencumbered software released into the public domain.
+
+Anyone is free to copy, modify, publish, use, compile, sell, or
+distribute this software, either in source code form or as a compiled
+binary, for any purpose, commercial or non-commercial, and by any
+means.
+
+In jurisdictions that recognize copyright laws, the author or authors
+of this software dedicate any and all copyright interest in the
+software to the public domain. We make this dedication for the benefit
+of the public at large and to the detriment of our heirs and
+successors. We intend this dedication to be an overt act of
+relinquishment in perpetuity of all present and future rights to this
+software under copyright law.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS BE LIABLE FOR ANY CLAIM, DAMAGES OR
+OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+For more information, please refer to <http://unlicense.org/>
+
+-----
+
 The following software may be included in this product: color-string. A copy of the source code may be downloaded from https://github.com/Qix-/color-string.git. This software contains the following license and notice below:
 
 Copyright (c) 2011 Heather Arthur <fayearthur@gmail.com>
@@ -4833,6 +5520,30 @@
 
 -----
 
+The following software may be included in this product: colormap. A copy of the source code may be downloaded from https://github.com/bpostlethwaite/colormap.git. This software contains the following license and notice below:
+
+Copyright (c) <2012> ICRL
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: combined-stream, delayed-stream. A copy of the source code may be downloaded from git://github.com/felixge/node-combined-stream.git (combined-stream), git://github.com/felixge/node-delayed-stream.git (delayed-stream). This software contains the following license and notice below:
 
 Copyright (c) 2011 Debuggable Limited <felix@debuggable.com>
@@ -4998,26 +5709,29 @@
 
 -----
 
-The following software may be included in this product: concat-map, is-typedarray, json-stable-stringify, json-stable-stringify-without-jsonify, minimist, path-browserify, resolve, safe-regex, semver-compare, text-table, tty-browserify, wordwrap. A copy of the source code may be downloaded from git://github.com/substack/node-concat-map.git (concat-map), git://github.com/hughsk/is-typedarray.git (is-typedarray), git://github.com/substack/json-stable-stringify.git (json-stable-stringify), git://github.com/samn/json-stable-stringify.git (json-stable-stringify-without-jsonify), git://github.com/substack/minimist.git (minimist), git://github.com/substack/path-browserify.git (path-browserify), git://github.com/substack/node-resolve.git (resolve), git://github.com/substack/safe-regex.git (safe-regex), git://github.com/substack/semver-compare.git (semver-compare), git://github.com/substack/text-table.git (text-table), git://github.com/substack/tty-browserify.git (tty-browserify), git://github.com/substack/node-wordwrap.git (wordwrap). This software contains the following license and notice below:
+The following software may be included in this product: compute-dims. A copy of the source code may be downloaded from git://github.com/compute-io/dims.git. This software contains the following license and notice below:
 
-This software is released under the MIT license:
+The MIT License (MIT)
 
-Permission is hereby granted, free of charge, to any person obtaining a copy of
-this software and associated documentation files (the "Software"), to deal in
-the Software without restriction, including without limitation the rights to
-use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
-the Software, and to permit persons to whom the Software is furnished to do so,
-subject to the following conditions:
+Copyright (c) 2014-2015 The Compute.io Authors. All rights reserved.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
 
 The above copyright notice and this permission notice shall be included in all
 copies or substantial portions of the Software.
 
 THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
-FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
-COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
-IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
-CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
 
 -----
 
@@ -5106,6 +5820,58 @@
 
 -----
 
+The following software may be included in this product: const-max-uint32. A copy of the source code may be downloaded from git://github.com/const-io/max-uint32.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015-2016 The Compute.io Authors.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: const-pinf-float64. A copy of the source code may be downloaded from git://github.com/const-io/pinf-float64.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 The Compute.io Authors.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: content-disposition, depd, forwarded, vary. A copy of the source code may be downloaded from https://github.com/jshttp/content-disposition.git (content-disposition), https://github.com/dougwilson/nodejs-depd.git (depd), https://github.com/jshttp/forwarded.git (forwarded), https://github.com/jshttp/vary.git (vary). This software contains the following license and notice below:
 
 (The MIT License)
@@ -5188,6 +5954,32 @@
 
 -----
 
+The following software may be included in this product: convex-hull, gl-surface3d, glsl-inverse, glsl-specular-beckmann, glsl-specular-cook-torrance, robust-in-sphere. A copy of the source code may be downloaded from https://github.com/mikolalysenko/convex-hull.git (convex-hull), git://github.com/gl-vis/gl-surface3d.git (gl-surface3d), git://github.com/stackgl/glsl-inverse.git (glsl-inverse), git://github.com/stackgl/glsl-specular-beckmann.git (glsl-specular-beckmann), git://github.com/stackgl/glsl-specular-cook-torrance.git (glsl-specular-cook-torrance), git://github.com/mikolalysenko/robust-in-sphere.git (robust-in-sphere). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: cookie. A copy of the source code may be downloaded from https://github.com/jshttp/cookie.git. This software contains the following license and notice below:
 
 (The MIT License)
@@ -5435,6 +6227,32 @@
 
 -----
 
+The following software may be included in this product: country-regex. A copy of the source code may be downloaded from git+https://github.com/etpinard/country-regex.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2017  Étienne Tétreault-Pinard
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: create-ecdh. A copy of the source code may be downloaded from https://github.com/crypto-browserify/createECDH.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -5579,6 +6397,32 @@
 
 -----
 
+The following software may be included in this product: css-font, css-font-size-keywords, css-font-stretch-keywords, css-font-style-keywords, css-font-weight-keywords, css-global-keywords, css-system-font-keywords. A copy of the source code may be downloaded from git+https://github.com/dy/css-font.git (css-font), git+https://github.com/jedmao/css-font-size-keywords.git (css-font-size-keywords), git+https://github.com/jedmao/css-font-stretch-keywords.git (css-font-stretch-keywords), git+https://github.com/jedmao/css-font-style-keywords.git (css-font-style-keywords), git+https://github.com/jedmao/css-font-weight-keywords.git (css-font-weight-keywords), git+https://github.com/jedmao/css-global-keywords.git (css-global-keywords), git+https://github.com/jedmao/css-system-font-keywords.git (css-system-font-keywords). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Jed Mao
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: css-loader, enhanced-resolve, eslint-loader, file-loader, html-webpack-plugin, less-loader, loader-utils, memory-fs, mini-css-extract-plugin, schema-utils, style-loader, terser-webpack-plugin, url-loader, watchpack, webpack, webpack-dev-middleware, webpack-dev-server. A copy of the source code may be downloaded from https://github.com/webpack-contrib/css-loader.git (css-loader), git://github.com/webpack/enhanced-resolve.git (enhanced-resolve), https://github.com/webpack-contrib/eslint-loader.git (eslint-loader), https://github.com/webpack-contrib/file-loader.git (file-loader), https://github.com/jantimon/html-webpack-plugin.git (html-webpack-plugin), https://github.com/webpack-contrib/less-loader.git (less-loader), https://github.com/webpack/loader-utils.git (loader-utils), https://github.com/webpack/memory-fs.git (memory-fs), https://github.com/webpack-contrib/mini-css-extract-plugin.git (mini-css-extract-plugin), https://github.com/webpack-contrib/schema-utils (schema-utils), https://github.com/webpack-contrib/style-loader.git (style-loader), https://github.com/webpack-contrib/terser-webpack-plugin.git (terser-webpack-plugin), https://github.com/webpack-contrib/url-loader.git (url-loader), https://github.com/webpack/watchpack.git (watchpack), https://github.com/webpack/webpack.git (webpack), https://github.com/webpack/webpack-dev-middleware.git (webpack-dev-middleware), https://github.com/webpack/webpack-dev-server.git (webpack-dev-server). This software contains the following license and notice below:
 
 Copyright JS Foundation and other contributors
@@ -5964,6 +6808,165 @@
 
 -----
 
+The following software may be included in this product: d3. A copy of the source code may be downloaded from https://github.com/mbostock/d3.git. This software contains the following license and notice below:
+
+Copyright (c) 2010-2016, Michael Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* The name Michael Bostock may not be used to endorse or promote products
+  derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT,
+INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
+OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: d3-array, d3-color, d3-dispatch, d3-force, d3-hierarchy, d3-interpolate, d3-quadtree, d3-timer. A copy of the source code may be downloaded from https://github.com/d3/d3-array.git (d3-array), https://github.com/d3/d3-color.git (d3-color), https://github.com/d3/d3-dispatch.git (d3-dispatch), https://github.com/d3/d3-force.git (d3-force), https://github.com/d3/d3-hierarchy.git (d3-hierarchy), https://github.com/d3/d3-interpolate.git (d3-interpolate), https://github.com/d3/d3-quadtree.git (d3-quadtree), https://github.com/d3/d3-timer.git (d3-timer). This software contains the following license and notice below:
+
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: d3-collection. A copy of the source code may be downloaded from https://github.com/d3/d3-collection.git. This software contains the following license and notice below:
+
+Copyright 2010-2016, Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: d3-path. A copy of the source code may be downloaded from https://github.com/d3/d3-path.git. This software contains the following license and notice below:
+
+Copyright 2015-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: d3-shape. A copy of the source code may be downloaded from https://github.com/d3/d3-shape.git. This software contains the following license and notice below:
+
+Copyright 2010-2015 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: damerau-levenshtein. A copy of the source code may be downloaded from https://github.com/tad-lispy/node-damerau-levenshtein.git. This software contains the following license and notice below:
 
 BSD 2-Clause License
@@ -6676,6 +7679,32 @@
 
 -----
 
+The following software may be included in this product: dom-helpers. A copy of the source code may be downloaded from git+https://github.com/react-bootstrap/dom-helpers.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Jason Quense
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: dom-serializer. A copy of the source code may be downloaded from git://github.com/cheeriojs/dom-renderer.git. This software contains the following license and notice below:
 
 License
@@ -6838,7 +7867,7 @@
 
 -----
 
-The following software may be included in this product: duplexer3. A copy of the source code may be downloaded from https://github.com/floatdrop/duplexer3.git. This software contains the following license and notice below:
+The following software may be included in this product: duplexer2, duplexer3. A copy of the source code may be downloaded from git://github.com/deoxxa/duplexer2.git (duplexer2), https://github.com/floatdrop/duplexer3.git (duplexer3). This software contains the following license and notice below:
 
 Copyright (c) 2013, Deoxxa Development
 ======================================
@@ -6869,7 +7898,7 @@
 
 -----
 
-The following software may be included in this product: duplexify, end-of-stream, pump, pumpify. A copy of the source code may be downloaded from git://github.com/mafintosh/duplexify (duplexify), git://github.com/mafintosh/end-of-stream.git (end-of-stream), git://github.com/mafintosh/pump.git (pump), git://github.com/mafintosh/pumpify (pumpify). This software contains the following license and notice below:
+The following software may be included in this product: duplexify, end-of-stream, protocol-buffers-schema, pump, pumpify. A copy of the source code may be downloaded from git://github.com/mafintosh/duplexify (duplexify), git://github.com/mafintosh/end-of-stream.git (end-of-stream), https://github.com/mafintosh/protocol-buffers-schema (protocol-buffers-schema), git://github.com/mafintosh/pump.git (pump), git://github.com/mafintosh/pumpify (pumpify). This software contains the following license and notice below:
 
 The MIT License (MIT)
 
@@ -6895,6 +7924,26 @@
 
 -----
 
+The following software may be included in this product: earcut, supercluster. A copy of the source code may be downloaded from git://github.com/mapbox/earcut.git (earcut), git://github.com/mapbox/supercluster.git (supercluster). This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2016, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: ecc-jsbn. A copy of the source code may be downloaded from https://github.com/quartzjer/ecc-jsbn.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -6931,6 +7980,32 @@
 
 -----
 
+The following software may be included in this product: elementary-circuits-directed-graph. A copy of the source code may be downloaded from git://github.com/antoinerg/elementary-circuits-directed-graph.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2019 Antoine Roy-Gobeil
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: emojis-list. A copy of the source code may be downloaded from git+https://github.com/kikobeats/emojis-list.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -7143,6 +8218,50 @@
 
 -----
 
+The following software may be included in this product: es6-promise, rsvp. A copy of the source code may be downloaded from git://github.com/stefanpenner/es6-promise.git (es6-promise), https://github.com/tildeio/rsvp.js.git (rsvp). This software contains the following license and notice below:
+
+Copyright (c) 2014 Yehuda Katz, Tom Dale, Stefan Penner and contributors
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: es6-weak-map. A copy of the source code may be downloaded from git://github.com/medikoo/es6-weak-map.git. This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2013-2018, Mariusz Nowak, @medikoo, medikoo.com
+
+Permission to use, copy, modify, and/or distribute this software for any
+purpose with or without fee is hereby granted, provided that the above
+copyright notice and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+PERFORMANCE OF THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: escape-html. A copy of the source code may be downloaded from https://github.com/component/escape-html.git. This software contains the following license and notice below:
 
 (The MIT License)
@@ -7172,7 +8291,7 @@
 
 -----
 
-The following software may be included in this product: escodegen, estraverse, esutils. A copy of the source code may be downloaded from http://github.com/estools/escodegen.git (escodegen), http://github.com/estools/estraverse.git (estraverse), http://github.com/estools/esutils.git (esutils). This software contains the following license and notice below:
+The following software may be included in this product: escodegen, esprima, estraverse, esutils. A copy of the source code may be downloaded from http://github.com/estools/escodegen.git (escodegen), http://github.com/ariya/esprima.git (esprima), http://github.com/Constellation/estraverse.git (estraverse), http://github.com/estools/esutils.git (esutils). This software contains the following license and notice below:
 
 Redistribution and use in source and binary forms, with or without
 modification, are permitted provided that the following conditions are met:
@@ -8105,6 +9224,32 @@
 
 -----
 
+The following software may be included in this product: fast-isnumeric, is-string-blank. A copy of the source code may be downloaded from git+https://github.com/plotly/fast-isnumeric.git (fast-isnumeric), git+https://github.com/plotly/is-string-blank.git (is-string-blank). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Alex Johnson
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: fast-json-stable-stringify. A copy of the source code may be downloaded from git://github.com/epoberezkin/fast-json-stable-stringify.git. This software contains the following license and notice below:
 
 This software is released under the MIT license:
@@ -8255,6 +9400,38 @@
 
 The following software may be included in this product: filesize. A copy of the source code may be downloaded from git://github.com/avoidwork/filesize.js.git. This software contains the following license and notice below:
 
+Copyright (c) 2020, Jason Mulligan
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of filesize nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: filesize. A copy of the source code may be downloaded from git://github.com/avoidwork/filesize.js.git. This software contains the following license and notice below:
+
 Copyright (c) 2019, Jason Mulligan
 All rights reserved.
 
@@ -8410,6 +9587,31 @@
 
 -----
 
+The following software may be included in this product: flatten-vertex-data. A copy of the source code may be downloaded from git://github.com/glo-js/flatten-vertex-data.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright (c) 2015 glo-js
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: follow-redirects. A copy of the source code may be downloaded from git@github.com:follow-redirects/follow-redirects.git. This software contains the following license and notice below:
 
 Copyright 2014–present Olivier Lalonde <olalonde@gmail.com>, James Talmage <james@talmage.io>, Ruben Verborgh
@@ -8433,6 +9635,32 @@
 
 -----
 
+The following software may be included in this product: for-each. A copy of the source code may be downloaded from git://github.com/Raynos/for-each.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2012 Raynos.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: for-own. A copy of the source code may be downloaded from https://github.com/jonschlinkert/for-own.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -8459,6 +9687,35 @@
 
 -----
 
+The following software may be included in this product: foreach. A copy of the source code may be downloaded from git://github.com/manuelstofer/foreach. This software contains the following license and notice below:
+
+The MIT License
+
+Copyright (c) 2013 Manuel Stofer
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: fork-ts-checker-webpack-plugin. A copy of the source code may be downloaded from https://github.com/TypeStrong/fork-ts-checker-webpack-plugin.git. This software contains the following license and notice below:
 
 MIT License
@@ -8563,7 +9820,7 @@
 
 -----
 
-The following software may be included in this product: from2. A copy of the source code may be downloaded from git://github.com/hughsk/from2. This software contains the following license and notice below:
+The following software may be included in this product: from2, glsl-resolve, map-limit. A copy of the source code may be downloaded from git://github.com/hughsk/from2 (from2), git://github.com/hughsk/glsl-resolve (glsl-resolve), git://github.com/hughsk/map-limit (map-limit). This software contains the following license and notice below:
 
 ## The MIT License (MIT) ##
 
@@ -8762,32 +10019,6 @@
 
 -----
 
-The following software may be included in this product: functional-red-black-tree, uniq. A copy of the source code may be downloaded from git://github.com/mikolalysenko/functional-red-black-tree.git (functional-red-black-tree), git://github.com/mikolalysenko/uniq.git (uniq). This software contains the following license and notice below:
-
-The MIT License (MIT)
-
-Copyright (c) 2013 Mikola Lysenko
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
-
------
-
 The following software may be included in this product: gensync. This software contains the following license and notice below:
 
 Copyright 2018 Logan Smyth <loganfsmyth@gmail.com>
@@ -8800,6 +10031,26 @@
 
 -----
 
+The following software may be included in this product: geojson-vt. A copy of the source code may be downloaded from git://github.com/mapbox/geojson-vt.git. This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2015, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: get-caller-file. A copy of the source code may be downloaded from git+https://github.com/stefanpenner/get-caller-file.git. This software contains the following license and notice below:
 
 ISC License (ISC)
@@ -8878,6 +10129,274 @@
 
 -----
 
+The following software may be included in this product: gl-axes3d. A copy of the source code may be downloaded from git://github.com/gl-vis/gl-axes3d.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: gl-cone3d, gl-streamtube3d. A copy of the source code may be downloaded from git://github.com/gl-vis/gl-cone3d.git (gl-cone3d), git://github.com/gl-vis/gl-streamtube3d.git (gl-streamtube3d). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2013-2017 Mikola Lysenko, Ilmari Heikkinen
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: gl-constants. A copy of the source code may be downloaded from git://github.com/mattdesl/gl-constants.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright (c) 2015 Matt DesLauriers
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: gl-format-compiler-error. A copy of the source code may be downloaded from git+https://github.com/wwwtyro/gl-format-compiler-error.git. This software contains the following license and notice below:
+
+This is free and unencumbered software released into the public domain.
+
+Anyone is free to copy, modify, publish, use, compile, sell, or
+distribute this software, either in source code form or as a compiled
+binary, for any purpose, commercial or non-commercial, and by any
+means.
+
+In jurisdictions that recognize copyright laws, the author or authors
+of this software dedicate any and all copyright interest in the
+software to the public domain. We make this dedication for the benefit
+of the public at large and to the detriment of our heirs and
+successors. We intend this dedication to be an overt act of
+relinquishment in perpetuity of all present and future rights to this
+software under copyright law.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS BE LIABLE FOR ANY CLAIM, DAMAGES OR
+OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+For more information, please refer to <http://unlicense.org>
+
+-----
+
+The following software may be included in this product: gl-mat2. A copy of the source code may be downloaded from git://github.com/gl-modules/gl-mat2.git. This software contains the following license and notice below:
+
+Copyright (c) 2013 Brandon Jones, Colin MacKenzie IV, Hugh Kennedy
+
+This software is provided 'as-is', without any express or implied
+warranty. In no event will the authors be held liable for any damages
+arising from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose,
+including commercial applications, and to alter it and redistribute it
+freely, subject to the following restrictions:
+
+ 1. The origin of this software must not be misrepresented; you must not
+    claim that you wrote the original software. If you use this software
+    in a product, an acknowledgment in the product documentation would be
+    appreciated but is not required.
+
+ 2. Altered source versions must be plainly marked as such, and must not
+    be misrepresented as being the original software.
+
+ 3. This notice may not be removed or altered from any source distribution.
+
+-----
+
+The following software may be included in this product: gl-mat3. A copy of the source code may be downloaded from git://github.com/gl-modules/gl-mat3.git. This software contains the following license and notice below:
+
+Copyright (c) 2013 Brandon Jones, Colin MacKenzie IV, Hugh Kennedy
+
+This software is provided 'as-is', without any express or implied
+warranty. In no event will the authors be held liable for any damages
+arising from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose,
+including commercial applications, and to alter it and redistribute it
+freely, subject to the following restrictions:
+
+ 1. The origin of this software must not be misrepresented you must not
+    claim that you wrote the original software. If you use this software
+    in a product, an acknowledgment in the product documentation would be
+    appreciated but is not required.
+
+ 2. Altered source versions must be plainly marked as such, and must not
+    be misrepresented as being the original software.
+
+ 3. This notice may not be removed or altered from any source distribution.
+
+-----
+
+The following software may be included in this product: gl-mat4, gl-vec3. A copy of the source code may be downloaded from https://github.com/stackgl/gl-mat4 (gl-mat4), git://github.com/stackgl/gl-vec3.git (gl-vec3). This software contains the following license and notice below:
+
+Copyright (c) 2013 Brandon Jones, Colin MacKenzie IV
+
+This software is provided 'as-is', without any express or implied
+warranty. In no event will the authors be held liable for any damages
+arising from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose,
+including commercial applications, and to alter it and redistribute it
+freely, subject to the following restrictions:
+
+ 1. The origin of this software must not be misrepresented you must not
+    claim that you wrote the original software. If you use this software
+    in a product, an acknowledgment in the product documentation would be
+    appreciated but is not required.
+
+ 2. Altered source versions must be plainly marked as such, and must not
+    be misrepresented as being the original software.
+
+ 3. This notice may not be removed or altered from any source distribution.
+
+-----
+
+The following software may be included in this product: gl-matrix. A copy of the source code may be downloaded from https://github.com/toji/gl-matrix.git. This software contains the following license and notice below:
+
+Copyright (c) 2015-2020, Brandon Jones, Colin MacKenzie IV.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: gl-plot3d. A copy of the source code may be downloaded from https://github.com/gl-vis/gl-plot3d.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: gl-quat, gl-vec4. A copy of the source code may be downloaded from git://github.com/stackgl/gl-quat.git (gl-quat), git://github.com/stackgl/gl-vec4.git (gl-vec4). This software contains the following license and notice below:
+
+Copyright (c) 2013 Brandon Jones, Colin MacKenzie IV
+
+This software is provided 'as-is', without any express or implied warranty. In no event will the authors be held liable for any damages arising from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose, including commercial applications, and to alter it and redistribute it freely, subject to the following restrictions:
+
+The origin of this software must not be misrepresented you must not claim that you wrote the original software. If you use this software in a product, an acknowledgment in the product documentation would be appreciated but is not required.
+
+Altered source versions must be plainly marked as such, and must not be misrepresented as being the original software.
+
+This notice may not be removed or altered from any source distribution.
+
+-----
+
+The following software may be included in this product: gl-text, regl-splom. A copy of the source code may be downloaded from git+https://github.com/a-vis/gl-text.git (gl-text), git+https://github.com/gl-vis/regl-splom.git (regl-splom). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2018 Dmitry Ivanov
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: glob. A copy of the source code may be downloaded from git://github.com/isaacs/node-glob.git. This software contains the following license and notice below:
 
 The ISC License
@@ -8970,6 +10489,119 @@
 
 -----
 
+The following software may be included in this product: glsl-inject-defines, glsl-shader-name, glsl-token-assignments, glsl-token-defines, glsl-token-depth, glsl-token-descope, glsl-token-properties, glsl-token-scope, glsl-token-string, glslify-bundle. A copy of the source code may be downloaded from git://github.com/mattdesl/glsl-inject-defines.git (glsl-inject-defines), git://github.com/stackgl/glsl-shader-name.git (glsl-shader-name), git://github.com/stackgl/glsl-token-assignments.git (glsl-token-assignments), git://github.com/stackgl/glsl-token-defines.git (glsl-token-defines), git://github.com/stackgl/glsl-token-depth.git (glsl-token-depth), git://github.com/stackgl/glsl-token-descope.git (glsl-token-descope), git://github.com/stackgl/glsl-token-properties.git (glsl-token-properties), git://github.com/stackgl/glsl-token-scope.git (glsl-token-scope), git://github.com/stackgl/glsl-token-string.git (glsl-token-string), git://github.com/stackgl/glslify-bundle.git (glslify-bundle). This software contains the following license and notice below:
+
+The MIT License (MIT)
+=====================
+
+Copyright (c) 2014 [stackgl](http://github.com/stackgl/) contributors
+
+*stackgl contributors listed at <https://github.com/stackgl/contributing/blob/master/CONTRIBUTING.md#contributors>*
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: glsl-out-of-range. A copy of the source code may be downloaded from git://github.com/plotly/glsl-out-of-range.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2019 Plotly, Inc
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: glsl-token-whitespace-trim. A copy of the source code may be downloaded from git://github.com/hughsk/glsl-token-whitespace-trim.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright (c) 2015 Hugh Kennedy
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: glsl-tokenizer. A copy of the source code may be downloaded from git://github.com/gl-modules/glsl-tokenizer.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+=====================
+
+Copyright (c) 2014 [Chris Dickinson](http://github.com/chrisdickinson)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: glslify. A copy of the source code may be downloaded from git://github.com/stackgl/glslify.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+=====================
+
+* Copyright &copy; 2015 [Chris Dickinson](http://github.com/chrisdickinson)
+* Copyright &copy; 2015 [stackgl](http://github.com/stackgl/) contributors
+
+*stackgl contributors listed at <https://github.com/stackgl/contributing/blob/master/CONTRIBUTING.md#contributors>*
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: graceful-fs. A copy of the source code may be downloaded from https://github.com/isaacs/node-graceful-fs. This software contains the following license and notice below:
 
 The ISC License
@@ -8990,6 +10622,24 @@
 
 -----
 
+The following software may be included in this product: grid-index. A copy of the source code may be downloaded from git+https://github.com/mapbox/grid-index.git. This software contains the following license and notice below:
+
+Copyright (c) 2016, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: hammerjs. A copy of the source code may be downloaded from git://github.com/hammerjs/hammer.js.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -9072,6 +10722,32 @@
 
 -----
 
+The following software may be included in this product: has-passive-events, parse-rect, point-cluster, pxls, regl-error2d, regl-line2d, regl-scatter2d, to-float32, to-uint8. A copy of the source code may be downloaded from git+https://github.com/dfcreative/has-passive-events.git (has-passive-events), git+https://github.com/dfcreative/parse-rect.git (parse-rect), git://github.com/dy/point-cluster.git (point-cluster), git+https://github.com/dy/pxls.git (pxls), git+https://github.com/dy/regl-error2d.git (regl-error2d), git+https://github.com/dy/regl-line2d.git (regl-line2d), git+https://github.com/dy/regl-scatter2d.git (regl-scatter2d), git+https://github.com/dy/to-float32.git (to-float32), git+https://github.com/dy/to-uint8.git (to-uint8). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2017 Dmitry Ivanov
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: has-symbols. A copy of the source code may be downloaded from git://github.com/ljharb/has-symbols.git. This software contains the following license and notice below:
 
 MIT License
@@ -10033,6 +11709,56 @@
 
 -----
 
+The following software may be included in this product: is-base64. A copy of the source code may be downloaded from https://github.com/miguelmota/is-base64. This software contains the following license and notice below:
+
+MIT license
+
+Copyright (C) 2015 Miguel Mota
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: is-browser, throat, uglify-to-browserify. A copy of the source code may be downloaded from https://github.com/ForbesLindesay/is-browser.git (is-browser), https://github.com/ForbesLindesay/throat.git (throat), https://github.com/ForbesLindesay/uglify-to-browserify.git (uglify-to-browserify). This software contains the following license and notice below:
+
+Copyright (c) 2013 Forbes Lindesay
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: is-buffer, safe-buffer, typedarray-to-buffer. A copy of the source code may be downloaded from git://github.com/feross/is-buffer.git (is-buffer), git://github.com/feross/safe-buffer.git (safe-buffer), git://github.com/feross/typedarray-to-buffer.git (typedarray-to-buffer). This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -10135,6 +11861,58 @@
 
 -----
 
+The following software may be included in this product: is-firefox. A copy of the source code may be downloaded from https://github.com/gillstrom/is-firefox.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) Andreas Gillström <andreasgillstrom@gmail.com> (github.com/gillstrom)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: is-iexplorer. A copy of the source code may be downloaded from https://github.com/kevva/is-iexplorer.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) Kevin Mårtensson <kevinmartensson@gmail.com>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: is-negated-glob. A copy of the source code may be downloaded from https://github.com/jonschlinkert/is-negated-glob.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -10691,6 +12469,26 @@
 
 -----
 
+The following software may be included in this product: kdbush, quickselect. A copy of the source code may be downloaded from git://github.com/mourner/kdbush.git (kdbush). This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2018, Vladimir Agafonkin
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: killable. A copy of the source code may be downloaded from https://github.com/marten-de-vries/killable.git. This software contains the following license and notice below:
 
 Copyright 2014 Marten de Vries
@@ -11326,6 +13124,18 @@
 
 -----
 
+The following software may be included in this product: magic-string. A copy of the source code may be downloaded from https://github.com/rich-harris/magic-string. This software contains the following license and notice below:
+
+Copyright 2018 Rich Harris
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: makeerror, tmpl. A copy of the source code may be downloaded from https://github.com/daaku/nodejs-makeerror (makeerror), https://github.com/daaku/nodejs-tmpl (tmpl). This software contains the following license and notice below:
 
 BSD License
@@ -11385,6 +13195,95 @@
 
 -----
 
+The following software may be included in this product: mapbox-gl. A copy of the source code may be downloaded from git://github.com/mapbox/mapbox-gl-js.git. This software contains the following license and notice below:
+
+Copyright (c) 2020, Mapbox
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+    * Neither the name of Mapbox GL JS nor the names of its contributors
+      may be used to endorse or promote products derived from this software
+      without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+-------------------------------------------------------------------------------
+
+Contains code from glfx.js
+
+Copyright (C) 2011 by Evan Wallace
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+Contains a portion of d3-color https://github.com/d3/d3-color
+
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: mdn-data, micro-spelling-correcter. A copy of the source code may be downloaded from https://github.com/mdn/data.git (mdn-data). This software contains the following license and notice below:
 
 CC0 1.0 Universal
@@ -11533,6 +13432,32 @@
 
 -----
 
+The following software may be included in this product: memoize-one, tiny-invariant, tiny-warning. A copy of the source code may be downloaded from https://github.com/alexreardon/memoize-one.git (memoize-one), https://github.com/alexreardon/tiny-invariant.git (tiny-invariant), https://github.com/alexreardon/tiny-warning.git (tiny-warning). This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2019 Alexander Reardon
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: memorystream. A copy of the source code may be downloaded from https://github.com/JSBizon/node-memorystream.git. This software contains the following license and notice below:
 
 Copyright (C) 2011 Dmitry Nizovtsev
@@ -11827,7 +13752,7 @@
 
 -----
 
-The following software may be included in this product: mkdirp, optimist. A copy of the source code may be downloaded from https://github.com/substack/node-mkdirp.git (mkdirp), http://github.com/substack/node-optimist.git (optimist). This software contains the following license and notice below:
+The following software may be included in this product: mkdirp, optimist, yargs. A copy of the source code may be downloaded from https://github.com/substack/node-mkdirp.git (mkdirp), http://github.com/substack/node-optimist.git (optimist), http://github.com/bcoe/yargs.git (yargs). This software contains the following license and notice below:
 
 Copyright 2010 James Halliday (mail@substack.net)
 
@@ -12027,6 +13952,32 @@
 
 -----
 
+The following software may be included in this product: ndarray. A copy of the source code may be downloaded from git://github.com/mikolalysenko/ndarray.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2013-2016 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: negotiator. A copy of the source code may be downloaded from https://github.com/jshttp/negotiator.git. This software contains the following license and notice below:
 
 (The MIT License)
@@ -12634,6 +14585,35 @@
 
 -----
 
+The following software may be included in this product: normalize-svg-path. A copy of the source code may be downloaded from git://github.com/jkroso/normalize-svg-path.git. This software contains the following license and notice below:
+
+The MIT License
+
+Copyright © 2008-2013 Dmitry Baranovskiy (http://raphaeljs.com)
+Copyright © 2008-2013 Sencha Labs (http://sencha.com)
+Copyright © 2013 Jake Rosoman <jkroso@gmail.com>
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+'Software'), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: num2fraction, postcss-media-minmax. A copy of the source code may be downloaded from git@github.com:yisibl/num2fraction.git (num2fraction), https://github.com/postcss/postcss-media-minmax.git (postcss-media-minmax). This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -13051,6 +15031,19 @@
 
 -----
 
+The following software may be included in this product: parenthesis. A copy of the source code may be downloaded from git://github.com/dy/parenthesis.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+Copyright © 2016 Dmitry Ivanov
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: parse-asn1. A copy of the source code may be downloaded from git://github.com/crypto-browserify/parse-asn1.git. This software contains the following license and notice below:
 
 Copyright (c) 2017, crypto-browserify contributors
@@ -13069,6 +15062,33 @@
 
 -----
 
+The following software may be included in this product: parse-svg-path. A copy of the source code may be downloaded from git://github.com/jkroso/parse-svg-path.git. This software contains the following license and notice below:
+
+The MIT License
+
+Copyright (c) 2013 Jake Rosoman <jkroso@gmail.com>
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+'Software'), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: parse5. A copy of the source code may be downloaded from git://github.com/inikulin/parse5.git. This software contains the following license and notice below:
 
 Copyright (c) 2013-2016 Ivan Nikulin (ifaaan@gmail.com, https://github.com/inikulin)
@@ -13225,6 +15245,38 @@
 
 -----
 
+The following software may be included in this product: pbf. A copy of the source code may be downloaded from git@github.com:mapbox/pbf.git. This software contains the following license and notice below:
+
+Copyright (c) 2017, Mapbox
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of pbf nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: pbkdf2. A copy of the source code may be downloaded from https://github.com/crypto-browserify/pbkdf2.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -13393,6 +15445,32 @@
 
 -----
 
+The following software may be included in this product: plotly.js. A copy of the source code may be downloaded from https://github.com/plotly/plotly.js.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2020 Plotly, Inc
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: pluralize. A copy of the source code may be downloaded from https://github.com/blakeembrey/pluralize.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -13419,6 +15497,60 @@
 
 -----
 
+The following software may be included in this product: polybooljs. A copy of the source code may be downloaded from https://github.com/voidqk/polybooljs. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 Sean Connelly (@voidqk, web: syntheti.cc)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: polytope-closest-point. A copy of the source code may be downloaded from git://github.com/mikolalysenko/polytope-closest-point.git. This software contains the following license and notice below:
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: portfinder. A copy of the source code may be downloaded from git@github.com:http-party/node-portfinder.git. This software contains the following license and notice below:
 
 node-portfinder
@@ -13882,6 +16014,26 @@
 
 -----
 
+The following software may be included in this product: potpack. A copy of the source code may be downloaded from git+https://github.com/mapbox/potpack.git. This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2018, Mapbox
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: prettier. A copy of the source code may be downloaded from https://github.com/prettier/prettier.git. This software contains the following license and notice below:
 
 Copyright © James Long and contributors
@@ -14133,6 +16285,31 @@
 
 -----
 
+The following software may be included in this product: quantize. A copy of the source code may be downloaded from git://github.com/olivierlesnicki/quantize.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014 Olivier Lesnicki
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: querystring, querystring-es3. A copy of the source code may be downloaded from git://github.com/Gozala/querystring.git (querystring), git://github.com/mike-spainhower/querystring.git (querystring-es3). This software contains the following license and notice below:
 
 Copyright 2012 Irakli Gozalishvili. All rights reserved.
@@ -14489,6 +16666,32 @@
 
 -----
 
+The following software may be included in this product: react-input-autosize. A copy of the source code may be downloaded from https://github.com/JedWatson/react-input-autosize.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2018 Jed Watson
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: react-lazy-load. A copy of the source code may be downloaded from https://github.com/loktar00/react-lazy-load.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -14541,6 +16744,67 @@
 
 -----
 
+The following software may be included in this product: react-plotly.js. A copy of the source code may be downloaded from https://github.com/plotly/react-plotly.js.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2017 Plotly, Inc.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
+The following software may be included in this product: react-transition-group. A copy of the source code may be downloaded from https://github.com/reactjs/react-transition-group.git. This software contains the following license and notice below:
+
+BSD 3-Clause License
+
+Copyright (c) 2018, React Community
+Forked from React (https://github.com/facebook/react) Copyright 2013-present, Facebook, Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the copyright holder nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: readable-stream. A copy of the source code may be downloaded from git://github.com/nodejs/readable-stream. This software contains the following license and notice below:
 
 Node.js is licensed for use as follows:
@@ -14749,6 +17013,32 @@
 
 -----
 
+The following software may be included in this product: regex-regex, utils-regex-from-string. A copy of the source code may be downloaded from git://github.com/kgryte/regex-regex.git (regex-regex), git://github.com/kgryte/utils-regex-from-string.git (utils-regex-from-string). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: regexp-tree. A copy of the source code may be downloaded from https://github.com/DmitrySoshnikov/regexp-tree.git. This software contains the following license and notice below:
 
 MIT License
@@ -14880,6 +17170,32 @@
 
 -----
 
+The following software may be included in this product: regl. A copy of the source code may be downloaded from git+https://github.com/regl-project/regl.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 Mikola Lysenko
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: relateurl. A copy of the source code may be downloaded from git://github.com/stevenvachon/relateurl.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -15058,7 +17374,7 @@
 
 -----
 
-The following software may be included in this product: resolve. A copy of the source code may be downloaded from git://github.com/browserify/resolve.git. This software contains the following license and notice below:
+The following software may be included in this product: resolve, tape. A copy of the source code may be downloaded from git://github.com/browserify/resolve.git (resolve), git://github.com/substack/tape.git (tape). This software contains the following license and notice below:
 
 MIT License
 
@@ -15110,6 +17426,32 @@
 
 -----
 
+The following software may be included in this product: resolve-protobuf-schema. A copy of the source code may be downloaded from https://github.com/mafintosh/resolve-protobuf-schema.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014 Mathias Buus
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: resolve-url, urix. A copy of the source code may be downloaded from https://github.com/lydell/resolve-url.git (resolve-url), https://github.com/lydell/urix.git (urix). This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -15236,30 +17578,6 @@
 
 -----
 
-The following software may be included in this product: rsvp. A copy of the source code may be downloaded from https://github.com/tildeio/rsvp.js.git. This software contains the following license and notice below:
-
-Copyright (c) 2014 Yehuda Katz, Tom Dale, Stefan Penner and contributors
-
-Permission is hereby granted, free of charge, to any person obtaining a copy of
-this software and associated documentation files (the "Software"), to deal in
-the Software without restriction, including without limitation the rights to
-use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
-of the Software, and to permit persons to whom the Software is furnished to do
-so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all
-copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
-
------
-
 The following software may be included in this product: run-async. A copy of the source code may be downloaded from https://github.com/SBoudrias/run-async.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -15286,6 +17604,37 @@
 
 -----
 
+The following software may be included in this product: rw. A copy of the source code may be downloaded from http://github.com/mbostock/rw.git. This software contains the following license and notice below:
+
+Copyright (c) 2014-2016, Michael Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* The name Michael Bostock may not be used to endorse or promote products
+  derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT,
+INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
+OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: rxjs. A copy of the source code may be downloaded from https://github.com/reactivex/rxjs.git. This software contains the following license and notice below:
 
 Apache License
@@ -15546,6 +17895,32 @@
 
 -----
 
+The following software may be included in this product: sane-topojson. A copy of the source code may be downloaded from git+https://github.com/etpinard/sane-topojson.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015  Étienne Tétreault-Pinard
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: sass-loader. A copy of the source code may be downloaded from https://github.com/webpack-contrib/sass-loader.git. This software contains the following license and notice below:
 
 Copyright JS Foundation and other contributors
@@ -16292,6 +18667,32 @@
 
 -----
 
+The following software may be included in this product: sourcemap-codec. A copy of the source code may be downloaded from https://github.com/Rich-Harris/sourcemap-codec. This software contains the following license and notice below:
+
+The MIT License
+
+Copyright (c) 2015 Rich Harris
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: spdx-expression-parse. A copy of the source code may be downloaded from https://github.com/jslicense/spdx-expression-parse.js.git. This software contains the following license and notice below:
 
 The MIT License
@@ -16321,6 +18722,35 @@
 
 The following software may be included in this product: sprintf-js. A copy of the source code may be downloaded from https://github.com/alexei/sprintf.js.git. This software contains the following license and notice below:
 
+Copyright (c) 2007-present, Alexandru Mărășteanu <hello@alexei.ro>
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+* Redistributions of source code must retain the above copyright
+  notice, this list of conditions and the following disclaimer.
+* Redistributions in binary form must reproduce the above copyright
+  notice, this list of conditions and the following disclaimer in the
+  documentation and/or other materials provided with the distribution.
+* Neither the name of this software nor the names of its contributors may be
+  used to endorse or promote products derived from this software without
+  specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: sprintf-js. A copy of the source code may be downloaded from https://github.com/alexei/sprintf.js.git. This software contains the following license and notice below:
+
 Copyright (c) 2007-2014, Alexandru Marasteanu <hello [at) alexei (dot] ro>
 All rights reserved.
 
@@ -16348,6 +18778,30 @@
 
 -----
 
+The following software may be included in this product: stack-trace. A copy of the source code may be downloaded from git://github.com/felixge/node-stack-trace.git. This software contains the following license and notice below:
+
+Copyright (c) 2011 Felix Geisendörfer (felix@debuggable.com)
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in
+ all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+ THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: stack-utils. A copy of the source code may be downloaded from https://github.com/tapjs/stack-utils.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -16376,6 +18830,31 @@
 
 -----
 
+The following software may be included in this product: static-eval. A copy of the source code may be downloaded from git://github.com/browserify/static-eval.git. This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2013 James Halliday
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: statuses. A copy of the source code may be downloaded from https://github.com/jshttp/statuses.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -16556,6 +19035,31 @@
 
 -----
 
+The following software may be included in this product: string_decoder. A copy of the source code may be downloaded from git://github.com/rvagg/string_decoder.git. This software contains the following license and notice below:
+
+Copyright Joyent, Inc. and other Node contributors.
+
+Permission is hereby granted, free of charge, to any person obtaining a
+copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to permit
+persons to whom the Software is furnished to do so, subject to the
+following conditions:
+
+The above copyright notice and this permission notice shall be included
+in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
+OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
+NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE
+USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: string-convert. A copy of the source code may be downloaded from https://github.com/akiran/string-convert. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -16608,6 +19112,32 @@
 
 -----
 
+The following software may be included in this product: string.prototype.trim. A copy of the source code may be downloaded from git://github.com/es-shims/String.prototype.trim.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Jordan Harband
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: stringify-object. A copy of the source code may be downloaded from https://github.com/yeoman/stringify-object.git. This software contains the following license and notice below:
 
 Copyright (c) 2015, Yeoman team
@@ -16635,6 +19165,27 @@
 
 -----
 
+The following software may be included in this product: svg-arc-to-cubic-bezier. A copy of the source code may be downloaded from https://github.com/colinmeinke/svg-arc-to-cubic-bezier. This software contains the following license and notice below:
+
+Internet Systems Consortium license
+===================================
+
+Copyright (c) `2017`, `Colin Meinke`
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: svgo. A copy of the source code may be downloaded from git://github.com/svg/svgo.git. This software contains the following license and notice below:
 
 The MIT License
@@ -16784,30 +19335,6 @@
 
 -----
 
-The following software may be included in this product: throat. A copy of the source code may be downloaded from https://github.com/ForbesLindesay/throat.git. This software contains the following license and notice below:
-
-Copyright (c) 2013 Forbes Lindesay
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
-
------
-
 The following software may be included in this product: through2. A copy of the source code may be downloaded from https://github.com/rvagg/through2.git. This software contains the following license and notice below:
 
 # The MIT License (MIT)
@@ -16822,6 +19349,50 @@
 
 -----
 
+The following software may be included in this product: through2. A copy of the source code may be downloaded from https://github.com/rvagg/through2.git. This software contains the following license and notice below:
+
+Copyright 2013, Rod Vagg (the "Original Author")
+All rights reserved.
+
+MIT +no-false-attribs License
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+Distributions of all or part of the Software intended to be used
+by the recipients as they would use the unmodified Software,
+containing modifications that substantially alter, remove, or
+disable functionality of the Software, outside of the documented
+configuration mechanisms provided by the Software, shall be
+modified such that the Original Author's bug reporting email
+addresses and urls are either replaced with the contact information
+of the parties responsible for the changes, or removed entirely.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+
+Except where noted, this license applies to any and all software
+programs and associated documentation files created by the
+Original Author, when distributed with the Software.
+
+-----
+
 The following software may be included in this product: thunky. A copy of the source code may be downloaded from git://github.com/mafintosh/thunky.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -16925,32 +19496,6 @@
 
 -----
 
-The following software may be included in this product: tiny-invariant, tiny-warning. A copy of the source code may be downloaded from https://github.com/alexreardon/tiny-invariant.git (tiny-invariant), https://github.com/alexreardon/tiny-warning.git (tiny-warning). This software contains the following license and notice below:
-
-MIT License
-
-Copyright (c) 2019 Alexander Reardon
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all
-copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
-
------
-
 The following software may be included in this product: tinycolor2. A copy of the source code may be downloaded from https://bgrins.github.com/TinyColor. This software contains the following license and notice below:
 
 Copyright (c), Brian Grinstead, http://briangrinstead.com
@@ -16976,6 +19521,26 @@
 
 -----
 
+The following software may be included in this product: tinyqueue. A copy of the source code may be downloaded from https://github.com/mourner/tinyqueue.git. This software contains the following license and notice below:
+
+ISC License
+
+Copyright (c) 2017, Vladimir Agafonkin
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: tmp. A copy of the source code may be downloaded from https://github.com/raszi/node-tmp.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -17072,6 +19637,24 @@
 
 -----
 
+The following software may be included in this product: topojson-client. A copy of the source code may be downloaded from https://github.com/topojson/topojson-client.git. This software contains the following license and notice below:
+
+Copyright 2012-2019 Michael Bostock
+
+Permission to use, copy, modify, and/or distribute this software for any purpose
+with or without fee is hereby granted, provided that the above copyright notice
+and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS
+OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER
+TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+THIS SOFTWARE.
+
+-----
+
 The following software may be included in this product: tough-cookie. A copy of the source code may be downloaded from git://github.com/salesforce/tough-cookie.git. This software contains the following license and notice below:
 
 Copyright (c) 2015, Salesforce.com, Inc.
@@ -17267,6 +19850,20 @@
 
 -----
 
+The following software may be included in this product: type-name. A copy of the source code may be downloaded from git://github.com/twada/type-name.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014-2016 Takuto Wada, https://github.com/twada/type-name
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----
+
 The following software may be included in this product: typedarray. A copy of the source code may be downloaded from git://github.com/substack/typedarray.git. This software contains the following license and notice below:
 
 /*
@@ -17337,6 +19934,40 @@
 
 UglifyJS is released under the BSD license:
 
+Copyright 2012-2013 (c) Mihai Bazon <mihai.bazon@gmail.com>
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+    * Redistributions of source code must retain the above
+      copyright notice, this list of conditions and the following
+      disclaimer.
+
+    * Redistributions in binary form must reproduce the above
+      copyright notice, this list of conditions and the following
+      disclaimer in the documentation and/or other materials
+      provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER “AS IS” AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER BE
+LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
+OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR
+TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF
+THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+SUCH DAMAGE.
+
+-----
+
+The following software may be included in this product: uglify-js. A copy of the source code may be downloaded from https://github.com/mishoo/UglifyJS2.git. This software contains the following license and notice below:
+
+UglifyJS is released under the BSD license:
+
 Copyright 2012-2019 (c) Mihai Bazon <mihai.bazon@gmail.com>
 
 Redistribution and use in source and binary forms, with or without
@@ -17625,6 +20256,84 @@
 
 -----
 
+The following software may be included in this product: utils-copy, validate.io-array-like, validate.io-string-primitive. A copy of the source code may be downloaded from git://github.com/kgryte/utils-copy.git (utils-copy), git://github.com/validate-io/array-like.git (validate.io-array-like), git://github.com/validate-io/string-primitive.git (validate.io-string-primitive). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015-2016 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: utils-copy-error, validate.io-buffer, validate.io-integer-primitive, validate.io-matrix-like, validate.io-ndarray-like, validate.io-nonnegative-integer, validate.io-number-primitive, validate.io-positive-integer. A copy of the source code may be downloaded from git://github.com/kgryte/utils-copy-error.git (utils-copy-error), git://github.com/validate-io/buffer.git (validate.io-buffer), git://github.com/validate-io/integer-primitive.git (validate.io-integer-primitive), git://github.com/validate-io/matrix-like.git (validate.io-matrix-like), git://github.com/validate-io/ndarray-like.git (validate.io-ndarray-like), git://github.com/validate-io/nonnegative-integer.git (validate.io-nonnegative-integer), git://github.com/validate-io/number-primitive.git (validate.io-number-primitive), git://github.com/validate-io/positive-integer.git (validate.io-positive-integer). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: utils-indexof. A copy of the source code may be downloaded from git://github.com/kgryte/utils-indexof.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2016 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: utils-merge. A copy of the source code may be downloaded from git://github.com/jaredhanson/utils-merge.git. This software contains the following license and notice below:
 
 The MIT License (MIT)
@@ -17702,6 +20411,58 @@
 
 -----
 
+The following software may be included in this product: validate.io-array. A copy of the source code may be downloaded from git://github.com/validate-io/array.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014-2015 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
+The following software may be included in this product: validate.io-integer, validate.io-number. A copy of the source code may be downloaded from git://github.com/validate-io/integer.git (validate.io-integer), git://github.com/validate-io/number.git (validate.io-number). This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2014 Athan Reines.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: vendors. A copy of the source code may be downloaded from https://github.com/wooorm/vendors.git. This software contains the following license and notice below:
 
 (The MIT License)
@@ -17776,6 +20537,64 @@
 
 -----
 
+The following software may be included in this product: vt-pbf. A copy of the source code may be downloaded from git+https://github.com/mapbox/vt-pbf.git. This software contains the following license and notice below:
+
+The MIT License (MIT)
+
+Copyright (c) 2015 Anand Thakker
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+Contains geojson_wrapper.js from https://github.com/mapbox/mapbox-gl-js
+
+Copyright (c) 2014, Mapbox
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+    * Neither the name of Mapbox GL JS nor the names of its contributors
+      may be used to endorse or promote products derived from this software
+      without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+-----
+
 The following software may be included in this product: walker. A copy of the source code may be downloaded from https://github.com/daaku/nodejs-walker. This software contains the following license and notice below:
 
 Copyright 2013 Naitik Shah
@@ -18019,6 +20838,32 @@
 
 -----
 
+The following software may be included in this product: world-calendars. A copy of the source code may be downloaded from git+https://github.com/alexcjohnson/world-calendars.git. This software contains the following license and notice below:
+
+MIT License
+
+Copyright (c) 2016 alexcjohnson
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-----
+
 The following software may be included in this product: write-file-atomic. A copy of the source code may be downloaded from git://github.com/npm/write-file-atomic.git. This software contains the following license and notice below:
 
 Copyright (c) 2015, Rebecca Turner
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json
index 77d3a69..82fae37 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json
@@ -18,6 +18,7 @@
     "datanodes": [
       {
         "hostname": "localhost1.storage.enterprise.com",
+        "uuid": "b590734e-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574728876059,
         "storageReport": {
@@ -44,6 +45,7 @@
       },
       {
         "hostname": "localhost2.storage.enterprise.com",
+        "uuid": "b5907812-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574724876059,
         "storageReport": {
@@ -70,6 +72,7 @@
       },
       {
         "hostname": "localhost3.storage.enterprise.com",
+        "uuid": "b5907934-a5f2-11ea-bb37-0242ac130002",
         "state": "STALE",
         "lastHeartbeat": 1343544879843,
         "storageReport": {
@@ -102,6 +105,7 @@
       },
       {
         "hostname": "localhost4.storage.enterprise.com",
+        "uuid": "b5907a06-a5f2-11ea-bb37-0242ac130002",
         "state": "DEAD",
         "lastHeartbeat": 1074724876059,
         "storageReport": {
@@ -115,6 +119,7 @@
       },
       {
         "hostname": "localhost5.storage.enterprise.com",
+        "uuid": "b5907ac4-a5f2-11ea-bb37-0242ac130002",
         "state": "DECOMMISSIONING",
         "lastHeartbeat": 1574724876059,
         "storageReport": {
@@ -141,6 +146,7 @@
       },
       {
         "hostname": "localhost6.storage.enterprise.com",
+        "uuid": "b5907b82-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574724876059,
         "storageReport": {
@@ -167,6 +173,7 @@
       },
       {
         "hostname": "localhost7.storage.enterprise.com",
+        "uuid": "b5907c40-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574724876059,
         "storageReport": {
@@ -199,6 +206,7 @@
       },
       {
         "hostname": "localhost8.storage.enterprise.com",
+        "uuid": "b5907cf4-a5f2-11ea-bb37-0242ac130002",
         "state": "DECOMMISSIONED",
         "lastHeartbeat": 1574724876059,
         "storageReport": {
@@ -225,6 +233,7 @@
       },
       {
         "hostname": "localhost9.storage.enterprise.com",
+        "uuid": "b5907f4c-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574724874011,
         "storageReport": {
@@ -251,6 +260,7 @@
       },
       {
         "hostname": "localhost10.storage.enterprise.com",
+        "uuid": "b590801e-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574723876959,
         "storageReport": {
@@ -283,6 +293,7 @@
       },
       {
         "hostname": "localhost11.storage.enterprise.com",
+        "uuid": "b59080e6-a5f2-11ea-bb37-0242ac130002",
         "state": "STALE",
         "lastHeartbeat": 1474724876783,
         "storageReport": {
@@ -309,6 +320,7 @@
       },
       {
         "hostname": "localhost12.storage.enterprise.com",
+        "uuid": "b59081a4-a5f2-11ea-bb37-0242ac130002",
         "state": "HEALTHY",
         "lastHeartbeat": 1574724796532,
         "storageReport": {
@@ -506,5 +518,55 @@
         "ModificationTime": "2019-11-26T21:19:22.251Z"
       }
     ]
-  }
+  },
+  "fileSizeCounts": [
+    {
+      "volume": "vol1",
+      "bucket": "bucket1",
+      "fileSize": 1024,
+      "count": 56
+    },
+    {
+      "volume": "vol1",
+      "bucket": "bucket1",
+      "fileSize": 4096,
+      "count": 25
+    },
+    {
+      "volume": "vol1",
+      "bucket": "bucket1",
+      "fileSize": 32768,
+      "count": 312
+    },
+    {
+      "volume": "vol1",
+      "bucket": "bucket1",
+      "fileSize": 131072,
+      "count": 78
+    },
+    {
+      "volume": "vol1",
+      "bucket": "bucket2",
+      "fileSize": 1024,
+      "count": 123
+    },
+    {
+      "volume": "vol1",
+      "bucket": "bucket2",
+      "fileSize": 131072,
+      "count": 187
+    },
+    {
+      "volume": "vol2",
+      "bucket": "bucket1",
+      "fileSize": 1024,
+      "count": 54
+    },
+    {
+      "volume": "vol2",
+      "bucket": "bucket2",
+      "fileSize": 131072,
+      "count": 217
+    }
+  ]
 }
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/routes.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/routes.json
index 0e9f320..0047e39 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/routes.json
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/routes.json
@@ -1,5 +1,6 @@
 {
   "/api/v1/*": "/$1",
   "/containers/:id/keys": "/keys",
-  "/containers/missing": "/missingContainers"
+  "/containers/missing": "/missingContainers",
+  "/utilization/fileCount": "/fileSizeCounts"
 }
\ No newline at end of file
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
index d9ac275..e663866 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
@@ -16,18 +16,20 @@
     "babel-plugin-import": "^1.11.0",
     "classnames": "^2.2.6",
     "customize-cra": "^0.2.12",
-    "filesize": "^6.0.1",
+    "filesize": "^6.1.0",
     "less": "^3.9.0",
     "less-loader": "^5.0.0",
     "moment": "^2.24.0",
-    "pretty-bytes": "^5.3.0",
+    "plotly.js": "^1.54.1",
     "pretty-ms": "^5.1.0",
     "react": "^16.8.6",
     "react-app-rewired": "^2.1.3",
     "react-dom": "^16.8.6",
+    "react-plotly.js": "^2.4.0",
     "react-router": "^5.1.2",
     "react-router-dom": "^5.0.0",
     "react-scripts": "^3.1.2",
+    "react-select": "^3.0.3",
     "typescript": "3.4.5"
   },
   "scripts": {
@@ -56,6 +58,8 @@
     ]
   },
   "devDependencies": {
+    "@types/react-plotly.js": "^2.2.4",
+    "@types/react-select": "^3.0.13",
     "@typescript-eslint/eslint-plugin": "^2.31.0",
     "@typescript-eslint/parser": "^2.31.0",
     "eslint": "^6.6.0",
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/multiSelect/multiSelect.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/multiSelect/multiSelect.tsx
new file mode 100644
index 0000000..19005dd
--- /dev/null
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/multiSelect/multiSelect.tsx
@@ -0,0 +1,123 @@
+/*
+ * 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.
+ */
+
+import React, {PureComponent} from 'react';
+import {
+  ActionMeta,
+  default as ReactSelect,
+  Props as ReactSelectProps,
+  ValueType,
+  components,
+  ValueContainerProps, OptionProps
+} from 'react-select';
+import makeAnimated from 'react-select/animated';
+
+export interface IOption {
+  label: string;
+  value: string;
+}
+
+interface IMultiSelectProps extends ReactSelectProps<IOption> {
+  options: IOption[];
+  allowSelectAll: boolean;
+  allOption?: IOption;
+}
+
+const defaultProps = {
+  allOption: {
+    label: 'Select all',
+    value: '*'
+  }
+};
+
+export class MultiSelect extends PureComponent<IMultiSelectProps> {
+  static defaultProps = defaultProps;
+  render() {
+    const {allowSelectAll, allOption, options, onChange} = this.props;
+    if (allowSelectAll) {
+      const Option = (props: OptionProps<IOption>) => {
+        return (
+          <div>
+            <components.Option {...props}>
+              <input
+                type='checkbox'
+                checked={props.isSelected}
+                onChange={() => null}
+              />{' '}
+              <label>{props.label}</label>
+            </components.Option>
+          </div>
+        );
+      };
+
+      const ValueContainer = ({children, ...props}: ValueContainerProps<IOption>) => {
+        const currentValues: IOption[] = props.getValue() as IOption[];
+        let toBeRendered = children;
+        if (currentValues.some(val => val.value === allOption!.value) && children) {
+          toBeRendered = allOption!.label;
+        } else if (currentValues.length >= 5) {
+          toBeRendered = `${currentValues.length} selected`;
+        }
+
+        return (
+          <components.ValueContainer {...props}>
+            {toBeRendered}
+          </components.ValueContainer>
+        );
+      };
+
+      const animatedComponents = makeAnimated();
+      return (
+        <ReactSelect
+          {...this.props}
+          options={[allOption!, ...options]}
+          components={{
+            Option,
+            ValueContainer,
+            animatedComponents
+          }}
+          onChange={(selected: ValueType<IOption>, event: ActionMeta<IOption>) => {
+            const selectedValues = selected as IOption[];
+            if (selectedValues && selectedValues.length > 0) {
+              if (selectedValues[selectedValues.length - 1].value === allOption!.value) {
+                return onChange!([allOption!, ...options], {action: 'select-option'});
+              }
+
+              let result: IOption[] = [];
+              if (selectedValues.length === options.length) {
+                if (selectedValues.some(option => option.value === allOption!.value)) {
+                  result = selectedValues.filter(
+                    option => option.value !== allOption!.value
+                  );
+                } else if (event.action === 'select-option') {
+                  result = [allOption!, ...options];
+                }
+
+                return onChange!(result, {action: 'select-option'});
+              }
+            }
+
+            return onChange!(selected, {action: 'select-option'});
+          }}
+        />
+      );
+    }
+
+    return <ReactSelect {...this.props}/>;
+  }
+}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/navBar/navBar.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/navBar/navBar.tsx
index b60a73f..a8e154a 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/navBar/navBar.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/navBar/navBar.tsx
@@ -66,6 +66,11 @@
             <span>Pipelines</span>
             <Link to='/Pipelines'/>
           </Menu.Item>
+          <Menu.Item key='/Insights'>
+            <Icon type='bar-chart'/>
+            <span>Insights</span>
+            <Link to='/Insights'/>
+          </Menu.Item>
         </Menu>
       </Sider>
     );
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/constants/breadcrumbs.constants.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/constants/breadcrumbs.constants.tsx
index 6e7a7d0..9f49b93 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/constants/breadcrumbs.constants.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/constants/breadcrumbs.constants.tsx
@@ -24,5 +24,6 @@
   '/Overview': 'Overview',
   '/Datanodes': 'Datanodes',
   '/Pipelines': 'Pipelines',
-  '/MissingContainers': 'Missing Containers'
+  '/MissingContainers': 'Missing Containers',
+  '/Insights': 'Insights'
 };
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/routes.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/routes.tsx
index 999efd8..a32040a 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/routes.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/routes.tsx
@@ -22,6 +22,7 @@
 import {NotFound} from './views/notFound/notFound';
 import {IRoute} from './types/routes.types';
 import {MissingContainers} from './views/missingContainers/missingContainers';
+import {Insights} from './views/insights/insights';
 
 export const routes: IRoute[] = [
   {
@@ -37,6 +38,10 @@
     component: Pipelines
   },
   {
+    path: '/Insights',
+    component: Insights
+  },
+  {
     path: '/MissingContainers',
     component: MissingContainers
   },
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
index d53b77b..bfba82a 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
@@ -135,10 +135,11 @@
           {
             pipelines.map((pipeline, index) => (
               <div key={index} className='pipeline-container'>
-                <ReplicationIcon replicationFactor={pipeline.replicationFactor}
-                                 replicationType={pipeline.replicationType}
-                                 leaderNode={pipeline.leaderNode}
-                                 isLeader={pipeline.leaderNode === record.hostname}/>
+                <ReplicationIcon
+                  replicationFactor={pipeline.replicationFactor}
+                  replicationType={pipeline.replicationType}
+                  leaderNode={pipeline.leaderNode}
+                  isLeader={pipeline.leaderNode === record.hostname}/>
                 {pipeline.pipelineID}
               </div>
             ))
@@ -148,12 +149,13 @@
     }
   },
   {
-    title: <span>
-      Leader Count&nbsp;
-      <Tooltip title='The number of Ratis Pipelines in which the given datanode is elected as a leader.'>
-        <Icon type='info-circle'/>
-      </Tooltip>
-    </span>,
+    title:
+  <span>
+    Leader Count&nbsp;
+    <Tooltip title='The number of Ratis Pipelines in which the given datanode is elected as a leader.'>
+      <Icon type='info-circle'/>
+    </Tooltip>
+  </span>,
     dataIndex: 'leaderCount',
     key: 'leaderCount',
     sorter: (a: IDatanode, b: IDatanode) => a.leaderCount - b.leaderCount
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.less b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.less
new file mode 100644
index 0000000..5d0d924
--- /dev/null
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.less
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+.insights-container {
+  .filter-block {
+    display: inline-block;
+    margin-left: 25px;
+  }
+
+  h4 {
+    display: inline-block;
+  }
+
+  .multi-select-container {
+    padding-left: 10px;
+    margin-right: 10px;
+    display: inline-block;
+    width: 300px;
+  }
+}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.tsx
new file mode 100644
index 0000000..19f0082
--- /dev/null
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/insights/insights.tsx
@@ -0,0 +1,279 @@
+/*
+ * 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.
+ */
+
+import React from 'react';
+import axios from 'axios';
+import {Icon, Row, Col} from 'antd';
+import filesize from 'filesize';
+import {showDataFetchError} from 'utils/common';
+import Plot from 'react-plotly.js';
+import * as Plotly from 'plotly.js';
+import {MultiSelect, IOption} from 'components/multiSelect/multiSelect';
+import {ActionMeta, ValueType} from 'react-select';
+import './insights.less';
+
+const size = filesize.partial({standard: 'iec'});
+
+interface IFileCountResponse {
+  volume: string;
+  bucket: string;
+  fileSize: number;
+  count: number;
+}
+
+interface IInsightsState {
+  isLoading: boolean;
+  fileCountsResponse: IFileCountResponse[];
+  plotData: Plotly.Data[];
+  volumeBucketMap: Map<string, Set<string>>;
+  selectedVolumes: IOption[];
+  selectedBuckets: IOption[];
+  bucketOptions: IOption[];
+  volumeOptions: IOption[];
+  isBucketSelectionDisabled: boolean;
+}
+
+const allVolumesOption: IOption = {
+  label: 'All Volumes',
+  value: '*'
+};
+
+const allBucketsOption: IOption = {
+  label: 'All Buckets',
+  value: '*'
+};
+
+export class Insights extends React.Component<Record<string, object>, IInsightsState> {
+  constructor(props = {}) {
+    super(props);
+    this.state = {
+      isLoading: false,
+      fileCountsResponse: [],
+      plotData: [],
+      volumeBucketMap: new Map<string, Set<string>>(),
+      selectedBuckets: [],
+      selectedVolumes: [],
+      bucketOptions: [],
+      volumeOptions: [],
+      isBucketSelectionDisabled: false
+    };
+  }
+
+  handleVolumeChange = (selected: ValueType<IOption>, _action: ActionMeta<IOption>) => {
+    const {volumeBucketMap} = this.state;
+    const selectedVolumes = (selected as IOption[]);
+
+    // Disable bucket selection dropdown if more than one volume is selected
+    // If there is only one volume, bucket selection dropdown should not be disabled.
+    const isBucketSelectionDisabled = !selectedVolumes ||
+        (selectedVolumes &&
+            (selectedVolumes.length > 2 &&
+                (volumeBucketMap.size !== 1)));
+    let bucketOptions: IOption[] = [];
+    // When volume is changed and more than one volume is selected,
+    // selected buckets value should be reset to all buckets
+    let selectedBuckets = [allBucketsOption];
+    // Update bucket options only if one volume is selected
+    if (selectedVolumes && selectedVolumes.length === 1) {
+      const selectedVolume = selectedVolumes[0].value;
+      if (volumeBucketMap.has(selectedVolume) && volumeBucketMap.get(selectedVolume)) {
+        bucketOptions = Array.from(volumeBucketMap.get(selectedVolume)!).map(bucket => ({
+          label: bucket,
+          value: bucket
+        }));
+        selectedBuckets = [...selectedBuckets, ...bucketOptions];
+      }
+    }
+
+    this.setState({
+      selectedVolumes,
+      selectedBuckets,
+      bucketOptions,
+      isBucketSelectionDisabled
+    }, this.updatePlotData);
+  };
+
+  handleBucketChange = (selected: ValueType<IOption>, _event: ActionMeta<IOption>) => {
+    const selectedBuckets = (selected as IOption[]);
+    this.setState({
+      selectedBuckets
+    }, this.updatePlotData);
+  };
+
+  updatePlotData = () => {
+    const {fileCountsResponse, selectedVolumes, selectedBuckets} = this.state;
+    // Aggregate counts across volumes & buckets
+    if (selectedVolumes && selectedBuckets) {
+      let filteredData = fileCountsResponse;
+      const selectedVolumeValues = new Set(selectedVolumes.map(option => option.value));
+      const selectedBucketValues = new Set(selectedBuckets.map(option => option.value));
+      if (selectedVolumes.length > 0 && !selectedVolumeValues.has(allVolumesOption.value)) {
+        // If not all volumes are selected, filter volumes based on the selection
+        filteredData = filteredData.filter(item => selectedVolumeValues.has(item.volume));
+      }
+
+      if (selectedBuckets.length > 0 && !selectedBucketValues.has(allBucketsOption.value)) {
+        // If not all buckets are selected, filter buckets based on the selection
+        filteredData = filteredData.filter(item => selectedBucketValues.has(item.bucket));
+      }
+
+      const xyMap: Map<number, number> = filteredData.reduce(
+        (map: Map<number, number>, current) => {
+          const fileSize = current.fileSize;
+          const oldCount = map.has(fileSize) ? map.get(fileSize)! : 0;
+          map.set(fileSize, oldCount + current.count);
+          return map;
+        }, new Map<number, number>());
+      // Calculate the previous power of 2 to find the lower bound of the range
+      // Ex: for 2048, the lower bound is 1024
+      const xValues = Array.from(xyMap.keys()).map(value => {
+        const upperbound = size(value);
+        const upperboundPower = Math.log2(value);
+        // For 1024 which is 2^10, the lowerbound is 0, since we start binning
+        // after 2^10
+        const lowerbound = upperboundPower > 10 ? size(2 ** (upperboundPower - 1)) : size(0);
+        return `${lowerbound} - ${upperbound}`;
+      });
+      this.setState({
+        plotData: [{
+          type: 'bar',
+          x: xValues,
+          y: Array.from(xyMap.values()),
+          name: 'file count'
+        }]
+      });
+    }
+  };
+
+  componentDidMount(): void {
+    // Fetch file size counts on component mount
+    this.setState({
+      isLoading: true
+    });
+    axios.get('/api/v1/utilization/fileCount').then(response => {
+      const fileCountsResponse: IFileCountResponse[] = response.data;
+      // Construct volume -> bucket[] map for populating filters
+      // Ex: vol1 -> [bucket1, bucket2], vol2 -> [bucket1]
+      const volumeBucketMap: Map<string, Set<string>> = fileCountsResponse.reduce(
+        (map: Map<string, Set<string>>, current) => {
+          const volume = current.volume;
+          const bucket = current.bucket;
+          if (map.has(volume)) {
+            const buckets = Array.from(map.get(volume)!);
+            map.set(volume, new Set([...buckets, bucket]));
+          } else {
+            map.set(volume, new Set().add(bucket));
+          }
+
+          return map;
+        }, new Map<string, Set<string>>());
+
+      // Set options for volume selection dropdown
+      const volumeOptions: IOption[] = Array.from(volumeBucketMap.keys()).map(k => ({
+        label: k,
+        value: k
+      }));
+
+      this.setState({
+        isLoading: false,
+        volumeBucketMap,
+        fileCountsResponse,
+        volumeOptions
+      }, () => {
+        this.updatePlotData();
+        // Select all volumes by default
+        this.handleVolumeChange([allVolumesOption, ...volumeOptions], {action: 'select-option'});
+      });
+    }).catch(error => {
+      this.setState({
+        isLoading: false
+      });
+      showDataFetchError(error.toString());
+    });
+  }
+
+  render() {
+    const {plotData, isLoading, selectedBuckets, volumeOptions,
+      selectedVolumes, fileCountsResponse, bucketOptions, isBucketSelectionDisabled} = this.state;
+    return (
+      <div className='insights-container'>
+        <div className='page-header'>
+          Insights
+        </div>
+        <div className='content-div'>
+          {isLoading ? <span><Icon type='loading'/> Loading...</span> :
+            ((fileCountsResponse && fileCountsResponse.length > 0) ?
+              <div>
+                <Row>
+                  <Col xs={24} xl={18}>
+                    <Row>
+                      <Col>
+                        <div className='filter-block'>
+                          <h4>Volumes</h4>
+                          <MultiSelect
+                            allowSelectAll
+                            isMulti
+                            className='multi-select-container'
+                            options={volumeOptions}
+                            closeMenuOnSelect={false}
+                            hideSelectedOptions={false}
+                            value={selectedVolumes}
+                            allOption={allVolumesOption}
+                            onChange={this.handleVolumeChange}
+                          />
+                        </div>
+                        <div className='filter-block'>
+                          <h4>Buckets</h4>
+                          <MultiSelect
+                            allowSelectAll
+                            isMulti
+                            className='multi-select-container'
+                            options={bucketOptions}
+                            closeMenuOnSelect={false}
+                            hideSelectedOptions={false}
+                            value={selectedBuckets}
+                            allOption={allBucketsOption}
+                            isDisabled={isBucketSelectionDisabled}
+                            onChange={this.handleBucketChange}
+                          />
+                        </div>
+                      </Col>
+                    </Row>
+                  </Col>
+                </Row>
+                <Row>
+                  <Col>
+                    <Plot
+                      data={plotData}
+                      layout={
+                        {
+                          width: 800,
+                          height: 600,
+                          title: 'File Size Distribution',
+                          showlegend: true
+                        }
+                      }/>
+                  </Col>
+                </Row>
+              </div> :
+              <div>No data to visualize file size distribution. Add files to Ozone to see a visualization on file size distribution.</div>)}
+        </div>
+      </div>
+    );
+  }
+}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/missingContainers/missingContainers.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/missingContainers/missingContainers.tsx
index bc87483..6a6dca3 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/missingContainers/missingContainers.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/missingContainers/missingContainers.tsx
@@ -19,11 +19,13 @@
 import React from 'react';
 import axios from 'axios';
 import {Icon, Table, Tooltip} from 'antd';
-import './missingContainers.less';
 import {PaginationConfig} from 'antd/lib/pagination';
-import prettyBytes from 'pretty-bytes';
+import filesize from 'filesize';
 import moment from 'moment';
-import {showDataFetchError, timeFormat} from '../../utils/common';
+import {showDataFetchError, timeFormat} from 'utils/common';
+import './missingContainers.less';
+
+const size = filesize.partial({standard: 'iec'});
 
 interface IMissingContainerResponse {
   containerID: number;
@@ -140,7 +142,7 @@
     title: 'Size',
     dataIndex: 'DataSize',
     key: 'DataSize',
-    render: (dataSize: number) => <div>{prettyBytes(dataSize)}</div>
+    render: (dataSize: number) => <div>{size(dataSize)}</div>
   },
   {
     title: 'Date Created',
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/overview/overview.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/overview/overview.tsx
index 7e84a0d..bd92f5f 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/overview/overview.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/overview/overview.tsx
@@ -20,14 +20,16 @@
 import {Row, Col, Icon, Tooltip} from 'antd';
 import OverviewCard from 'components/overviewCard/overviewCard';
 import axios from 'axios';
-import prettyBytes from 'pretty-bytes';
-import './overview.less';
 import {IStorageReport} from 'types/datanode.types';
 import {IMissingContainersResponse} from '../missingContainers/missingContainers';
 import moment from 'moment';
 import AutoReloadPanel from 'components/autoReloadPanel/autoReloadPanel';
 import {showDataFetchError} from 'utils/common';
 import {AutoReloadHelper} from 'utils/autoReloadHelper';
+import filesize from 'filesize';
+import './overview.less';
+
+const size = filesize.partial({round: 1});
 
 interface IClusterStateResponse {
   totalDatanodes: number;
@@ -137,7 +139,7 @@
       </span>
     ) :
       containers.toString();
-    const clusterCapacity = `${prettyBytes(storageReport.capacity - storageReport.remaining)}/${prettyBytes(storageReport.capacity)}`;
+    const clusterCapacity = `${size(storageReport.capacity - storageReport.remaining)}/${size(storageReport.capacity)}`;
     return (
       <div className='overview-content'>
         <div className='page-header'>
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
index f339060..15b6858 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
@@ -72,10 +72,11 @@
       const replicationFactor = record.replicationFactor;
       return (
         <span>
-          <ReplicationIcon replicationFactor={replicationFactor}
-                           replicationType={replicationType}
-                           leaderNode={record.leaderNode}
-                           isLeader={false}/>
+          <ReplicationIcon
+            replicationFactor={replicationFactor}
+            replicationType={replicationType}
+            leaderNode={record.leaderNode}
+            isLeader={false}/>
           {replicationType} ({replicationFactor})
         </span>
       );
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/yarn.lock b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/yarn.lock
index d7fb91a..bc1cea7 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/yarn.lock
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/yarn.lock
@@ -2,6 +2,15 @@
 # yarn lockfile v1
 
 
+"3d-view@^2.0.0":
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/3d-view/-/3d-view-2.0.0.tgz#831ae942d7508c50801e3e06fafe1e8c574e17be"
+  integrity sha1-gxrpQtdQjFCAHj4G+v4ejFdOF74=
+  dependencies:
+    matrix-camera-controller "^2.1.1"
+    orbit-camera-controller "^4.0.0"
+    turntable-camera-controller "^3.0.0"
+
 "@ant-design/colors@^3.1.0":
   version "3.2.2"
   resolved "https://registry.yarnpkg.com/@ant-design/colors/-/colors-3.2.2.tgz#5ad43d619e911f3488ebac303d606e66a8423903"
@@ -1165,6 +1174,13 @@
   dependencies:
     regenerator-runtime "^0.13.2"
 
+"@babel/runtime@^7.4.4", "@babel/runtime@^7.5.5", "@babel/runtime@^7.8.7":
+  version "7.10.2"
+  resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.10.2.tgz#d103f21f2602497d38348a32e008637d506db839"
+  integrity sha512-6sF3uQw2ivImfVIl62RZ7MXhO2tap69WeWK57vAaimT6AZbE4FbqjdEJIN1UqoD6wI6B+1n9UiagafH1sxjOtg==
+  dependencies:
+    regenerator-runtime "^0.13.4"
+
 "@babel/runtime@^7.4.5", "@babel/runtime@^7.7.2", "@babel/runtime@^7.8.4", "@babel/runtime@^7.9.2":
   version "7.9.6"
   resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.9.6.tgz#a9102eb5cadedf3f31d08a9ecf294af7827ea29f"
@@ -1238,6 +1254,13 @@
     lodash "^4.17.13"
     to-fast-properties "^2.0.0"
 
+"@choojs/findup@^0.2.0":
+  version "0.2.1"
+  resolved "https://registry.yarnpkg.com/@choojs/findup/-/findup-0.2.1.tgz#ac13c59ae7be6e1da64de0779a0a7f03d75615a3"
+  integrity sha512-YstAqNb0MCN8PjdLCDfRsBcGVRN41f3vgLvaI0IrIcBp4AqILRSS0DeWNGkicC+f/zRIPJLc+9RURVSepwvfBw==
+  dependencies:
+    commander "^2.15.1"
+
 "@cnakazawa/watch@^1.0.3":
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/@cnakazawa/watch/-/watch-1.0.3.tgz#099139eaec7ebf07a27c1786a3ff64f39464d2ef"
@@ -1256,6 +1279,83 @@
   resolved "https://registry.yarnpkg.com/@csstools/normalize.css/-/normalize.css-10.1.0.tgz#f0950bba18819512d42f7197e56c518aa491cf18"
   integrity sha512-ij4wRiunFfaJxjB0BdrYHIH8FxBJpOwNPhhAcunlmPdXudL1WQV1qoP9un6JsEBAgQH+7UXyyjh0g7jTxXK6tg==
 
+"@emotion/cache@^10.0.27", "@emotion/cache@^10.0.9":
+  version "10.0.29"
+  resolved "https://registry.yarnpkg.com/@emotion/cache/-/cache-10.0.29.tgz#87e7e64f412c060102d589fe7c6dc042e6f9d1e0"
+  integrity sha512-fU2VtSVlHiF27empSbxi1O2JFdNWZO+2NFHfwO0pxgTep6Xa3uGb+3pVKfLww2l/IBGLNEZl5Xf/++A4wAYDYQ==
+  dependencies:
+    "@emotion/sheet" "0.9.4"
+    "@emotion/stylis" "0.8.5"
+    "@emotion/utils" "0.11.3"
+    "@emotion/weak-memoize" "0.2.5"
+
+"@emotion/core@^10.0.9":
+  version "10.0.28"
+  resolved "https://registry.yarnpkg.com/@emotion/core/-/core-10.0.28.tgz#bb65af7262a234593a9e952c041d0f1c9b9bef3d"
+  integrity sha512-pH8UueKYO5jgg0Iq+AmCLxBsvuGtvlmiDCOuv8fGNYn3cowFpLN98L8zO56U0H1PjDIyAlXymgL3Wu7u7v6hbA==
+  dependencies:
+    "@babel/runtime" "^7.5.5"
+    "@emotion/cache" "^10.0.27"
+    "@emotion/css" "^10.0.27"
+    "@emotion/serialize" "^0.11.15"
+    "@emotion/sheet" "0.9.4"
+    "@emotion/utils" "0.11.3"
+
+"@emotion/css@^10.0.27", "@emotion/css@^10.0.9":
+  version "10.0.27"
+  resolved "https://registry.yarnpkg.com/@emotion/css/-/css-10.0.27.tgz#3a7458198fbbebb53b01b2b87f64e5e21241e14c"
+  integrity sha512-6wZjsvYeBhyZQYNrGoR5yPMYbMBNEnanDrqmsqS1mzDm1cOTu12shvl2j4QHNS36UaTE0USIJawCH9C8oW34Zw==
+  dependencies:
+    "@emotion/serialize" "^0.11.15"
+    "@emotion/utils" "0.11.3"
+    babel-plugin-emotion "^10.0.27"
+
+"@emotion/hash@0.8.0":
+  version "0.8.0"
+  resolved "https://registry.yarnpkg.com/@emotion/hash/-/hash-0.8.0.tgz#bbbff68978fefdbe68ccb533bc8cbe1d1afb5413"
+  integrity sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow==
+
+"@emotion/memoize@0.7.4":
+  version "0.7.4"
+  resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.7.4.tgz#19bf0f5af19149111c40d98bb0cf82119f5d9eeb"
+  integrity sha512-Ja/Vfqe3HpuzRsG1oBtWTHk2PGZ7GR+2Vz5iYGelAw8dx32K0y7PjVuxK6z1nMpZOqAFsRUPCkK1YjJ56qJlgw==
+
+"@emotion/serialize@^0.11.15", "@emotion/serialize@^0.11.16":
+  version "0.11.16"
+  resolved "https://registry.yarnpkg.com/@emotion/serialize/-/serialize-0.11.16.tgz#dee05f9e96ad2fb25a5206b6d759b2d1ed3379ad"
+  integrity sha512-G3J4o8by0VRrO+PFeSc3js2myYNOXVJ3Ya+RGVxnshRYgsvErfAOglKAiy1Eo1vhzxqtUvjCyS5gtewzkmvSSg==
+  dependencies:
+    "@emotion/hash" "0.8.0"
+    "@emotion/memoize" "0.7.4"
+    "@emotion/unitless" "0.7.5"
+    "@emotion/utils" "0.11.3"
+    csstype "^2.5.7"
+
+"@emotion/sheet@0.9.4":
+  version "0.9.4"
+  resolved "https://registry.yarnpkg.com/@emotion/sheet/-/sheet-0.9.4.tgz#894374bea39ec30f489bbfc3438192b9774d32e5"
+  integrity sha512-zM9PFmgVSqBw4zL101Q0HrBVTGmpAxFZH/pYx/cjJT5advXguvcgjHFTCaIO3enL/xr89vK2bh0Mfyj9aa0ANA==
+
+"@emotion/stylis@0.8.5":
+  version "0.8.5"
+  resolved "https://registry.yarnpkg.com/@emotion/stylis/-/stylis-0.8.5.tgz#deacb389bd6ee77d1e7fcaccce9e16c5c7e78e04"
+  integrity sha512-h6KtPihKFn3T9fuIrwvXXUOwlx3rfUvfZIcP5a6rh8Y7zjE3O06hT5Ss4S/YI1AYhuZ1kjaE/5EaOOI2NqSylQ==
+
+"@emotion/unitless@0.7.5":
+  version "0.7.5"
+  resolved "https://registry.yarnpkg.com/@emotion/unitless/-/unitless-0.7.5.tgz#77211291c1900a700b8a78cfafda3160d76949ed"
+  integrity sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg==
+
+"@emotion/utils@0.11.3":
+  version "0.11.3"
+  resolved "https://registry.yarnpkg.com/@emotion/utils/-/utils-0.11.3.tgz#a759863867befa7e583400d322652a3f44820924"
+  integrity sha512-0o4l6pZC+hI88+bzuaX/6BgOvQVhbt2PfmxauVaYOGgbsAw14wdKyvMCZXnsnsHys94iadcF+RG/wZyx6+ZZBw==
+
+"@emotion/weak-memoize@0.2.5":
+  version "0.2.5"
+  resolved "https://registry.yarnpkg.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz#8eed982e2ee6f7f4e44c253e12962980791efd46"
+  integrity sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA==
+
 "@hapi/address@2.x.x":
   version "2.1.4"
   resolved "https://registry.yarnpkg.com/@hapi/address/-/address-2.1.4.tgz#5d67ed43f3fd41a69d4b9ff7b56e7c0d1d0a81e5"
@@ -1436,6 +1536,56 @@
     "@types/istanbul-reports" "^1.1.1"
     "@types/yargs" "^13.0.0"
 
+"@mapbox/geojson-rewind@^0.5.0":
+  version "0.5.0"
+  resolved "https://registry.yarnpkg.com/@mapbox/geojson-rewind/-/geojson-rewind-0.5.0.tgz#91f0ad56008c120caa19414b644d741249f4f560"
+  integrity sha512-73l/qJQgj/T/zO1JXVfuVvvKDgikD/7D/rHAD28S9BG1OTstgmftrmqfCx4U+zQAmtsB6HcDA3a7ymdnJZAQgg==
+  dependencies:
+    concat-stream "~2.0.0"
+    minimist "^1.2.5"
+
+"@mapbox/geojson-types@^1.0.2":
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/@mapbox/geojson-types/-/geojson-types-1.0.2.tgz#9aecf642cb00eab1080a57c4f949a65b4a5846d6"
+  integrity sha512-e9EBqHHv3EORHrSfbR9DqecPNn+AmuAoQxV6aL8Xu30bJMJR1o8PZLZzpk1Wq7/NfCbuhmakHTPYRhoqLsXRnw==
+
+"@mapbox/jsonlint-lines-primitives@^2.0.2":
+  version "2.0.2"
+  resolved "https://registry.yarnpkg.com/@mapbox/jsonlint-lines-primitives/-/jsonlint-lines-primitives-2.0.2.tgz#ce56e539f83552b58d10d672ea4d6fc9adc7b234"
+  integrity sha1-zlblOfg1UrWNENZy6k1vya3HsjQ=
+
+"@mapbox/mapbox-gl-supported@^1.5.0":
+  version "1.5.0"
+  resolved "https://registry.yarnpkg.com/@mapbox/mapbox-gl-supported/-/mapbox-gl-supported-1.5.0.tgz#f60b6a55a5d8e5ee908347d2ce4250b15103dc8e"
+  integrity sha512-/PT1P6DNf7vjEEiPkVIRJkvibbqWtqnyGaBz3nfRdcxclNSnSdaLU5tfAgcD7I8Yt5i+L19s406YLl1koLnLbg==
+
+"@mapbox/point-geometry@0.1.0", "@mapbox/point-geometry@^0.1.0", "@mapbox/point-geometry@~0.1.0":
+  version "0.1.0"
+  resolved "https://registry.yarnpkg.com/@mapbox/point-geometry/-/point-geometry-0.1.0.tgz#8a83f9335c7860effa2eeeca254332aa0aeed8f2"
+  integrity sha1-ioP5M1x4YO/6Lu7KJUMyqgru2PI=
+
+"@mapbox/tiny-sdf@^1.1.1":
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/@mapbox/tiny-sdf/-/tiny-sdf-1.1.1.tgz#16a20c470741bfe9191deb336f46e194da4a91ff"
+  integrity sha512-Ihn1nZcGIswJ5XGbgFAvVumOgWpvIjBX9jiRlIl46uQG9vJOF51ViBYHF95rEZupuyQbEmhLaDPLQlU7fUTsBg==
+
+"@mapbox/unitbezier@^0.0.0":
+  version "0.0.0"
+  resolved "https://registry.yarnpkg.com/@mapbox/unitbezier/-/unitbezier-0.0.0.tgz#15651bd553a67b8581fb398810c98ad86a34524e"
+  integrity sha1-FWUb1VOme4WB+zmIEMmK2Go0Uk4=
+
+"@mapbox/vector-tile@^1.3.1":
+  version "1.3.1"
+  resolved "https://registry.yarnpkg.com/@mapbox/vector-tile/-/vector-tile-1.3.1.tgz#d3a74c90402d06e89ec66de49ec817ff53409666"
+  integrity sha512-MCEddb8u44/xfQ3oD+Srl/tNcQoqTw3goGk2oLsrFxOTc3dUp+kAnby3PvAeeBYSMSjSPD1nd1AJA6W49WnoUw==
+  dependencies:
+    "@mapbox/point-geometry" "~0.1.0"
+
+"@mapbox/whoots-js@^3.1.0":
+  version "3.1.0"
+  resolved "https://registry.yarnpkg.com/@mapbox/whoots-js/-/whoots-js-3.1.0.tgz#497c67a1cef50d1a2459ba60f315e448d2ad87fe"
+  integrity sha512-Es6WcD0nO5l+2BOQS4uLfNPYQaNDfbot3X1XUoloz+x0mPDS3eeORZJl06HXjwBG1fOGwCRnzK88LMdxKRrd6Q==
+
 "@mrmlnc/readdir-enhanced@^2.2.1":
   version "2.2.1"
   resolved "https://registry.yarnpkg.com/@mrmlnc/readdir-enhanced/-/readdir-enhanced-2.2.1.tgz#524af240d1a360527b730475ecfa1344aa540dde"
@@ -1449,6 +1599,25 @@
   resolved "https://registry.yarnpkg.com/@nodelib/fs.stat/-/fs.stat-1.1.3.tgz#2b5a3ab3f918cca48a8c754c08168e3f03eba61b"
   integrity sha512-shAmDyaQC4H92APFoIaVDHCx5bStIocgvbwQyxPRrbUY20V1EYTbSDchWbuwlMG3V17cprZhA6+78JfB+3DTPw==
 
+"@plotly/d3-sankey-circular@0.33.1":
+  version "0.33.1"
+  resolved "https://registry.yarnpkg.com/@plotly/d3-sankey-circular/-/d3-sankey-circular-0.33.1.tgz#15d1e0337e0e4b1135bdf0e2195c88adacace1a7"
+  integrity sha512-FgBV1HEvCr3DV7RHhDsPXyryknucxtfnLwPtCKKxdolKyTFYoLX/ibEfX39iFYIL7DYbVeRtP43dbFcrHNE+KQ==
+  dependencies:
+    d3-array "^1.2.1"
+    d3-collection "^1.0.4"
+    d3-shape "^1.2.0"
+    elementary-circuits-directed-graph "^1.0.4"
+
+"@plotly/d3-sankey@0.7.2":
+  version "0.7.2"
+  resolved "https://registry.yarnpkg.com/@plotly/d3-sankey/-/d3-sankey-0.7.2.tgz#ddd5290d3b02c60037ced018a162644a2ccef33b"
+  integrity sha512-2jdVos1N3mMp3QW0k2q1ph7Gd6j5PY1YihBrwpkFnKqO+cqtZq3AdEYUeSGXMeLsBDQYiqTVcihYfk8vr5tqhw==
+  dependencies:
+    d3-array "1"
+    d3-collection "1"
+    d3-shape "^1.2.0"
+
 "@sindresorhus/is@^0.14.0":
   version "0.14.0"
   resolved "https://registry.yarnpkg.com/@sindresorhus/is/-/is-0.14.0.tgz#9fb3a3cf3132328151f353de4632e01e52102bea"
@@ -1564,6 +1733,42 @@
   dependencies:
     defer-to-connect "^1.0.1"
 
+"@turf/area@^6.0.1":
+  version "6.0.1"
+  resolved "https://registry.yarnpkg.com/@turf/area/-/area-6.0.1.tgz#50ed63c70ef2bdb72952384f1594319d94f3b051"
+  integrity sha512-Zv+3N1ep9P5JvR0YOYagLANyapGWQBh8atdeR3bKpWcigVXFsEKNUw03U/5xnh+cKzm7yozHD6MFJkqQv55y0g==
+  dependencies:
+    "@turf/helpers" "6.x"
+    "@turf/meta" "6.x"
+
+"@turf/bbox@^6.0.1":
+  version "6.0.1"
+  resolved "https://registry.yarnpkg.com/@turf/bbox/-/bbox-6.0.1.tgz#b966075771475940ee1c16be2a12cf389e6e923a"
+  integrity sha512-EGgaRLettBG25Iyx7VyUINsPpVj1x3nFQFiGS3ER8KCI1MximzNLsam3eXRabqQDjyAKyAE1bJ4EZEpGvspQxw==
+  dependencies:
+    "@turf/helpers" "6.x"
+    "@turf/meta" "6.x"
+
+"@turf/centroid@^6.0.2":
+  version "6.0.2"
+  resolved "https://registry.yarnpkg.com/@turf/centroid/-/centroid-6.0.2.tgz#c4eb16b4bc60b692f74e1809cf9a7c4a4f5ba1cc"
+  integrity sha512-auyDauOtC4eddH7GC3CHFTDu2PKhpSeKCRhwhHhXtJqn2dWCJQNIoCeJRmfXRIbzCWhWvgvQafvvhq8HNvmvWw==
+  dependencies:
+    "@turf/helpers" "6.x"
+    "@turf/meta" "6.x"
+
+"@turf/helpers@6.x":
+  version "6.1.4"
+  resolved "https://registry.yarnpkg.com/@turf/helpers/-/helpers-6.1.4.tgz#d6fd7ebe6782dd9c87dca5559bda5c48ae4c3836"
+  integrity sha512-vJvrdOZy1ngC7r3MDA7zIGSoIgyrkWcGnNIEaqn/APmw+bVLF2gAW7HIsdTxd12s5wQMqEpqIQrmrbRRZ0xC7g==
+
+"@turf/meta@6.x":
+  version "6.0.2"
+  resolved "https://registry.yarnpkg.com/@turf/meta/-/meta-6.0.2.tgz#eb92951126d24a613ac1b7b99d733fcc20fd30cf"
+  integrity sha512-VA7HJkx7qF1l3+GNGkDVn2oXy4+QoLP6LktXAaZKjuT1JI0YESat7quUkbCMy4zP9lAUuvS4YMslLyTtr919FA==
+  dependencies:
+    "@turf/helpers" "6.x"
+
 "@types/babel__core@^7.1.0":
   version "7.1.3"
   resolved "https://registry.yarnpkg.com/@types/babel__core/-/babel__core-7.1.3.tgz#e441ea7df63cd080dfcd02ab199e6d16a735fc30"
@@ -1607,6 +1812,11 @@
   resolved "https://registry.yarnpkg.com/@types/color-name/-/color-name-1.1.1.tgz#1c1261bbeaa10a8055bbc5d8ab84b7b2afc846a0"
   integrity sha512-rr+OQyAjxze7GgWrSaJwydHStIhHq2lvY3BOC2Mj7KnzI7XK0Uw1TOOdI9lDoajEbSWLiYgoo4f1R51erQfhPQ==
 
+"@types/d3@^3":
+  version "3.5.43"
+  resolved "https://registry.yarnpkg.com/@types/d3/-/d3-3.5.43.tgz#e9b4992817e0b6c5efaa7d6e5bb2cee4d73eab58"
+  integrity sha512-t9ZmXOcpVxywRw86YtIC54g7M9puRh8hFedRvVfHKf5YyOP6pSxA0TvpXpfseXSCInoW4P7bggTrSDiUOs4g5w==
+
 "@types/eslint-visitor-keys@^1.0.0":
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/@types/eslint-visitor-keys/-/eslint-visitor-keys-1.0.0.tgz#1ee30d79544ca84d68d4b3cdb0af4f205663dd2d"
@@ -1695,6 +1905,13 @@
   resolved "https://registry.yarnpkg.com/@types/parse-json/-/parse-json-4.0.0.tgz#2f8bb441434d163b35fb8ffdccd7138927ffb8c0"
   integrity sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA==
 
+"@types/plotly.js@*":
+  version "1.50.12"
+  resolved "https://registry.yarnpkg.com/@types/plotly.js/-/plotly.js-1.50.12.tgz#dd0c296d1c68d0415d2ce7241a627a73926f897e"
+  integrity sha512-ZyXsV0XCGI/clfo+o04kQNY8SAFR/9k/eiSmooEtg9EENdpOo+Z74qQSYGkGlNWZcdDvFWpWdeOAM7BrJLn/LQ==
+  dependencies:
+    "@types/d3" "^3"
+
 "@types/prop-types@*":
   version "15.7.3"
   resolved "https://registry.yarnpkg.com/@types/prop-types/-/prop-types-15.7.3.tgz#2ab0d5da2e5815f94b0b9d4b95d1e5f243ab2ca7"
@@ -1705,6 +1922,13 @@
   resolved "https://registry.yarnpkg.com/@types/q/-/q-1.5.2.tgz#690a1475b84f2a884fd07cd797c00f5f31356ea8"
   integrity sha512-ce5d3q03Ex0sy4R14722Rmt6MT07Ua+k4FwDfdcToYJcMKNtRVQvJ6JCAPdAmAnbRb6CsX6aYb9m96NGod9uTw==
 
+"@types/react-dom@*":
+  version "16.9.8"
+  resolved "https://registry.yarnpkg.com/@types/react-dom/-/react-dom-16.9.8.tgz#fe4c1e11dfc67155733dfa6aa65108b4971cb423"
+  integrity sha512-ykkPQ+5nFknnlU6lDd947WbQ6TE3NNzbQAkInC2EKY1qeYdTKp7onFusmYZb+ityzx2YviqT6BXSu+LyWWJwcA==
+  dependencies:
+    "@types/react" "*"
+
 "@types/react-dom@16.8.4":
   version "16.8.4"
   resolved "https://registry.yarnpkg.com/@types/react-dom/-/react-dom-16.8.4.tgz#7fb7ba368857c7aa0f4e4511c4710ca2c5a12a88"
@@ -1712,6 +1936,14 @@
   dependencies:
     "@types/react" "*"
 
+"@types/react-plotly.js@^2.2.4":
+  version "2.2.4"
+  resolved "https://registry.yarnpkg.com/@types/react-plotly.js/-/react-plotly.js-2.2.4.tgz#360fcfcace64d0ef173cf0c7d67fc2441b39f659"
+  integrity sha512-dsvngno7Ar13XoF2eJXM609Msoc5DPboNp8x4NE4L+rsBEEsu16/0b0Fze8REx3fA4HEUk4rcn3uJsx48m2sew==
+  dependencies:
+    "@types/plotly.js" "*"
+    "@types/react" "*"
+
 "@types/react-router-dom@^4.3.3":
   version "4.3.5"
   resolved "https://registry.yarnpkg.com/@types/react-router-dom/-/react-router-dom-4.3.5.tgz#72f229967690c890d00f96e6b85e9ee5780db31f"
@@ -1729,6 +1961,15 @@
     "@types/history" "*"
     "@types/react" "*"
 
+"@types/react-select@^3.0.13":
+  version "3.0.13"
+  resolved "https://registry.yarnpkg.com/@types/react-select/-/react-select-3.0.13.tgz#b1a05eae0f65fb4f899b4db1f89b8420cb9f3656"
+  integrity sha512-JxmSArGgzAOtb37+Jz2+3av8rVmp/3s3DGwlcP+g59/a3owkiuuU4/Jajd+qA32beDPHy4gJR2kkxagPY3j9kg==
+  dependencies:
+    "@types/react" "*"
+    "@types/react-dom" "*"
+    "@types/react-transition-group" "*"
+
 "@types/react-slick@^0.23.4":
   version "0.23.4"
   resolved "https://registry.yarnpkg.com/@types/react-slick/-/react-slick-0.23.4.tgz#c97e2a9e7e3d1933c68593b8e82752fab1e8ce53"
@@ -1736,6 +1977,13 @@
   dependencies:
     "@types/react" "*"
 
+"@types/react-transition-group@*":
+  version "4.4.0"
+  resolved "https://registry.yarnpkg.com/@types/react-transition-group/-/react-transition-group-4.4.0.tgz#882839db465df1320e4753e6e9f70ca7e9b4d46d"
+  integrity sha512-/QfLHGpu+2fQOqQaXh8MG9q03bFENooTb/it4jr5kKaZlDQfWvjqWZg48AwzPVMBHlRuTRAY7hRHCEOXz5kV6w==
+  dependencies:
+    "@types/react" "*"
+
 "@types/react@*":
   version "16.9.17"
   resolved "https://registry.yarnpkg.com/@types/react/-/react-16.9.17.tgz#58f0cc0e9ec2425d1441dd7b623421a867aa253e"
@@ -1968,11 +2216,25 @@
   resolved "https://registry.yarnpkg.com/@xtuc/long/-/long-4.2.2.tgz#d291c6a4e97989b5c61d9acf396ae4fe133a718d"
   integrity sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ==
 
+a-big-triangle@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/a-big-triangle/-/a-big-triangle-1.0.3.tgz#eefd30b02a8f525e8b1f72bb6bb1b0c16751c794"
+  integrity sha1-7v0wsCqPUl6LH3K7a7GwwWdRx5Q=
+  dependencies:
+    gl-buffer "^2.1.1"
+    gl-vao "^1.2.0"
+    weak-map "^1.0.5"
+
 abab@^2.0.0:
   version "2.0.3"
   resolved "https://registry.yarnpkg.com/abab/-/abab-2.0.3.tgz#623e2075e02eb2d3f2475e49f99c91846467907a"
   integrity sha512-tsFzPpcttalNjFBCFMqsKYQcWxxen1pgJR56by//QwvJc4/OUS3kPOOttx2tSIfjsylB0pYu7f5D3K1RCxUnUg==
 
+abs-svg-path@^0.1.1, abs-svg-path@~0.1.1:
+  version "0.1.1"
+  resolved "https://registry.yarnpkg.com/abs-svg-path/-/abs-svg-path-0.1.1.tgz#df601c8e8d2ba10d4a76d625e236a9a39c2723bf"
+  integrity sha1-32Acjo0roQ1KdtYl4japo5wnI78=
+
 accepts@~1.3.4, accepts@~1.3.5, accepts@~1.3.7:
   version "1.3.7"
   resolved "https://registry.yarnpkg.com/accepts/-/accepts-1.3.7.tgz#531bc726517a3b2b41f850021c6cc15eaab507cd"
@@ -1981,6 +2243,11 @@
     mime-types "~2.1.24"
     negotiator "0.6.2"
 
+acorn-dynamic-import@^4.0.0:
+  version "4.0.0"
+  resolved "https://registry.yarnpkg.com/acorn-dynamic-import/-/acorn-dynamic-import-4.0.0.tgz#482210140582a36b83c3e342e1cfebcaa9240948"
+  integrity sha512-d3OEjQV4ROpoflsnUA8HozoIR504TFxNivYEUi6uwz0IYhBkTDXGuWlNdMtybRt3nqVx/L6XqMt0FxkXuWKZhw==
+
 acorn-globals@^4.1.0, acorn-globals@^4.3.0:
   version "4.3.4"
   resolved "https://registry.yarnpkg.com/acorn-globals/-/acorn-globals-4.3.4.tgz#9fa1926addc11c97308c4e66d7add0d40c3272e7"
@@ -1989,7 +2256,7 @@
     acorn "^6.0.1"
     acorn-walk "^6.0.1"
 
-acorn-jsx@^5.2.0:
+acorn-jsx@^5.0.1, acorn-jsx@^5.2.0:
   version "5.2.0"
   resolved "https://registry.yarnpkg.com/acorn-jsx/-/acorn-jsx-5.2.0.tgz#4c66069173d6fdd68ed85239fc256226182b2ebe"
   integrity sha512-HiUX/+K2YpkpJ+SzBffkM/AQ2YE03S0U1kjTLVpoJdhZMOWy8qvXVN9JdLqv2QsaQ6MPYQIuNmwD8zOiYUofLQ==
@@ -2009,7 +2276,7 @@
   resolved "https://registry.yarnpkg.com/acorn/-/acorn-6.4.0.tgz#b659d2ffbafa24baf5db1cdbb2c94a983ecd2784"
   integrity sha512-gac8OEcQ2Li1dxIEWGZzsp2BitJxwkwcOm0zHAJLcPJaVvm58FRnk6RkuLRpU1EujipU2ZFODv2P9DLMfnV8mw==
 
-acorn@^6.2.1:
+acorn@^6.1.1, acorn@^6.2.1:
   version "6.4.1"
   resolved "https://registry.yarnpkg.com/acorn/-/acorn-6.4.1.tgz#531e58ba3f51b9dacb9a6646ca4debf5b14ca474"
   integrity sha512-ZVA9k326Nwrj3Cj9jlh3wGFutC2ZornPNARZwsNYqQYgN0EsV2d53w5RN/co65Ohn4sUAUtb1rSUAOD6XN9idA==
@@ -2026,6 +2293,13 @@
   dependencies:
     object-assign "4.x"
 
+add-line-numbers@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/add-line-numbers/-/add-line-numbers-1.0.1.tgz#48dbbdea47dbd234deafeac6c93cea6f70b4b7e3"
+  integrity sha1-SNu96kfb0jTer+rGyTzqb3C0t+M=
+  dependencies:
+    pad-left "^1.0.2"
+
 address@1.1.2, address@^1.0.1:
   version "1.1.2"
   resolved "https://registry.yarnpkg.com/address/-/address-1.1.2.tgz#bf1116c9c758c51b7a933d296b72c221ed9428b6"
@@ -2042,6 +2316,13 @@
     object-path "0.11.4"
     regex-parser "2.2.10"
 
+affine-hull@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/affine-hull/-/affine-hull-1.0.0.tgz#763ff1d38d063ceb7e272f17ee4d7bbcaf905c5d"
+  integrity sha1-dj/x040GPOt+Jy8X7k17vK+QXF0=
+  dependencies:
+    robust-orientation "^1.1.3"
+
 aggregate-error@^3.0.0:
   version "3.0.1"
   resolved "https://registry.yarnpkg.com/aggregate-error/-/aggregate-error-3.0.1.tgz#db2fe7246e536f40d9b5442a39e117d7dd6a24e0"
@@ -2080,11 +2361,46 @@
     json-schema-traverse "^0.4.1"
     uri-js "^4.2.2"
 
+align-text@^0.1.1, align-text@^0.1.3:
+  version "0.1.4"
+  resolved "https://registry.yarnpkg.com/align-text/-/align-text-0.1.4.tgz#0cd90a561093f35d0a99256c22b7069433fad117"
+  integrity sha1-DNkKVhCT810KmSVsIrcGlDP60Rc=
+  dependencies:
+    kind-of "^3.0.2"
+    longest "^1.0.1"
+    repeat-string "^1.5.2"
+
+almost-equal@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/almost-equal/-/almost-equal-1.1.0.tgz#f851c631138757994276aa2efbe8dfa3066cccdd"
+  integrity sha1-+FHGMROHV5lCdqou++jfowZszN0=
+
+alpha-complex@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/alpha-complex/-/alpha-complex-1.0.0.tgz#90865870d6b0542ae73c0c131d4ef989669b72d2"
+  integrity sha1-kIZYcNawVCrnPAwTHU75iWabctI=
+  dependencies:
+    circumradius "^1.0.0"
+    delaunay-triangulate "^1.1.6"
+
+alpha-shape@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/alpha-shape/-/alpha-shape-1.0.0.tgz#c83109923ecfda667d2163fe4f26fe24726f64a9"
+  integrity sha1-yDEJkj7P2mZ9IWP+Tyb+JHJvZKk=
+  dependencies:
+    alpha-complex "^1.0.0"
+    simplicial-complex-boundary "^1.0.0"
+
 alphanum-sort@^1.0.0:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/alphanum-sort/-/alphanum-sort-1.0.2.tgz#97a1119649b211ad33691d9f9f486a8ec9fbe0a3"
   integrity sha1-l6ERlkmyEa0zaR2fn0hqjsn74KM=
 
+amdefine@>=0.0.4:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/amdefine/-/amdefine-1.0.1.tgz#4a5282ac164729e93619bcfd3ad151f817ce91f5"
+  integrity sha1-SlKCrBZHKek2Gbz9OtFR+BfOkfU=
+
 ansi-align@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/ansi-align/-/ansi-align-3.0.0.tgz#b536b371cf687caaef236c18d3e21fe3797467cb"
@@ -2270,6 +2586,11 @@
   resolved "https://registry.yarnpkg.com/arr-union/-/arr-union-3.1.0.tgz#e39b09aea9def866a8f206e288af63919bae39c4"
   integrity sha1-45sJrqne+Gao8gbiiK9jkZuuOcQ=
 
+array-bounds@^1.0.0, array-bounds@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/array-bounds/-/array-bounds-1.0.1.tgz#da11356b4e18e075a4f0c86e1f179a67b7d7ea31"
+  integrity sha512-8wdW3ZGk6UjMPJx/glyEt0sLzzwAE1bhToPsO1W2pbpR2gULyxe3BjSiuJFheP50T/GgODVPz2fuMUmIywt8cQ==
+
 array-equal@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/array-equal/-/array-equal-1.0.0.tgz#8c2a5ef2472fd9ea742b04c77a75093ba2757c93"
@@ -2304,6 +2625,23 @@
     es-abstract "^1.17.0"
     is-string "^1.0.5"
 
+array-normalize@^1.1.4:
+  version "1.1.4"
+  resolved "https://registry.yarnpkg.com/array-normalize/-/array-normalize-1.1.4.tgz#d75cec57383358af38efdf6a78071aa36ae4174c"
+  integrity sha512-fCp0wKFLjvSPmCn4F5Tiw4M3lpMZoHlCjfcs7nNzuj3vqQQ1/a8cgB9DXcpDSn18c+coLnaW7rqfcYCvKbyJXg==
+  dependencies:
+    array-bounds "^1.0.0"
+
+array-range@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/array-range/-/array-range-1.0.1.tgz#f56e46591843611c6a56f77ef02eda7c50089bfc"
+  integrity sha1-9W5GWRhDYRxqVvd+8C7afFAIm/w=
+
+array-rearrange@^2.2.2:
+  version "2.2.2"
+  resolved "https://registry.yarnpkg.com/array-rearrange/-/array-rearrange-2.2.2.tgz#fa1a2acf8d02e88dd0c9602aa0e06a79158b2283"
+  integrity sha512-UfobP5N12Qm4Qu4fwLDIi2v6+wZsSf6snYSxAMeKhrh37YGnNWZPRmVEKc/2wfms53TLQnzfpG8wCx2Y/6NG1w==
+
 array-tree-filter@^2.1.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/array-tree-filter/-/array-tree-filter-2.1.0.tgz#873ac00fec83749f255ac8dd083814b4f6329190"
@@ -2432,6 +2770,16 @@
   resolved "https://registry.yarnpkg.com/at-least-node/-/at-least-node-1.0.0.tgz#602cd4b46e844ad4effc92a8011a3c46e0238dc2"
   integrity sha512-+q/t7Ekv1EDY2l6Gda6LLiX14rU9TV20Wa3ofeQmwPFZbOMo9DXrLbOjFaaclkXKWidIaopwAObQDqwWtGUjqg==
 
+atob-lite@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/atob-lite/-/atob-lite-1.0.0.tgz#b88dca6006922b962094f7556826bab31c4a296b"
+  integrity sha1-uI3KYAaSK5YglPdVaCa6sxxKKWs=
+
+atob-lite@^2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/atob-lite/-/atob-lite-2.0.0.tgz#0fef5ad46f1bd7a8502c65727f0367d5ee43d696"
+  integrity sha1-D+9a1G8b16hQLGVyfwNn1e5D1pY=
+
 atob@^2.1.2:
   version "2.1.2"
   resolved "https://registry.yarnpkg.com/atob/-/atob-2.1.2.tgz#6d9517eb9e030d2436666651e86bd9f6f13533c9"
@@ -2535,6 +2883,22 @@
   dependencies:
     object.assign "^4.1.0"
 
+babel-plugin-emotion@^10.0.27:
+  version "10.0.33"
+  resolved "https://registry.yarnpkg.com/babel-plugin-emotion/-/babel-plugin-emotion-10.0.33.tgz#ce1155dcd1783bbb9286051efee53f4e2be63e03"
+  integrity sha512-bxZbTTGz0AJQDHm8k6Rf3RQJ8tX2scsfsRyKVgAbiUPUNIRtlK+7JxP+TAd1kRLABFxe0CFm2VdK4ePkoA9FxQ==
+  dependencies:
+    "@babel/helper-module-imports" "^7.0.0"
+    "@emotion/hash" "0.8.0"
+    "@emotion/memoize" "0.7.4"
+    "@emotion/serialize" "^0.11.16"
+    babel-plugin-macros "^2.0.0"
+    babel-plugin-syntax-jsx "^6.18.0"
+    convert-source-map "^1.5.0"
+    escape-string-regexp "^1.0.5"
+    find-root "^1.1.0"
+    source-map "^0.5.7"
+
 babel-plugin-import@^1.11.0:
   version "1.13.0"
   resolved "https://registry.yarnpkg.com/babel-plugin-import/-/babel-plugin-import-1.13.0.tgz#c532fd533df9db53b47d4d4db3676090fc5c07a5"
@@ -2560,7 +2924,7 @@
   dependencies:
     "@types/babel__traverse" "^7.0.6"
 
-babel-plugin-macros@2.8.0:
+babel-plugin-macros@2.8.0, babel-plugin-macros@^2.0.0:
   version "2.8.0"
   resolved "https://registry.yarnpkg.com/babel-plugin-macros/-/babel-plugin-macros-2.8.0.tgz#0f958a7cc6556b1e65344465d99111a1e5e10138"
   integrity sha512-SEP5kJpfGYqYKpBrj5XU3ahw5p5GOHJ0U5ssOSQ/WBVdwkD2Dzlce95exQTs3jOVWPPKLBN2rlEWkCK7dSmLvg==
@@ -2574,6 +2938,11 @@
   resolved "https://registry.yarnpkg.com/babel-plugin-named-asset-import/-/babel-plugin-named-asset-import-0.3.6.tgz#c9750a1b38d85112c9e166bf3ef7c5dbc605f4be"
   integrity sha512-1aGDUfL1qOOIoqk9QKGIo2lANk+C7ko/fqH0uIyC71x3PEGz0uVP8ISgfEsFuG+FKmjHTvFK/nNM8dowpmUxLA==
 
+babel-plugin-syntax-jsx@^6.18.0:
+  version "6.18.0"
+  resolved "https://registry.yarnpkg.com/babel-plugin-syntax-jsx/-/babel-plugin-syntax-jsx-6.18.0.tgz#0af32a9a6e13ca7a3fd5069e62d7b0f58d0d8946"
+  integrity sha1-CvMqmm4Tyno/1QaeYtew9Y0NiUY=
+
 babel-plugin-syntax-object-rest-spread@^6.8.0:
   version "6.13.0"
   resolved "https://registry.yarnpkg.com/babel-plugin-syntax-object-rest-spread/-/babel-plugin-syntax-object-rest-spread-6.13.0.tgz#fd6536f2bce13836ffa3a5458c4903a597bb3bf5"
@@ -2639,6 +3008,13 @@
   resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.0.tgz#89b4d199ab2bee49de164ea02b89ce462d71b767"
   integrity sha1-ibTRmasr7kneFk6gK4nORi1xt2c=
 
+barycentric@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/barycentric/-/barycentric-1.0.1.tgz#f1562bb891b26f4fec463a82eeda3657800ec688"
+  integrity sha1-8VYruJGyb0/sRjqC7to2V4AOxog=
+  dependencies:
+    robust-linear-solve "^1.0.0"
+
 base64-js@^1.0.2:
   version "1.3.1"
   resolved "https://registry.yarnpkg.com/base64-js/-/base64-js-1.3.1.tgz#58ece8cb75dd07e71ed08c736abc5fac4dbf8df1"
@@ -2676,6 +3052,15 @@
   dependencies:
     tweetnacl "^0.14.3"
 
+big-rat@^1.0.3:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/big-rat/-/big-rat-1.0.4.tgz#768d093bb57930dd18ed575c7fca27dc5391adea"
+  integrity sha1-do0JO7V5MN0Y7Vdcf8on3FORreo=
+  dependencies:
+    bit-twiddle "^1.0.2"
+    bn.js "^4.11.6"
+    double-bits "^1.1.1"
+
 big.js@^5.2.2:
   version "5.2.2"
   resolved "https://registry.yarnpkg.com/big.js/-/big.js-5.2.2.tgz#65f0af382f578bcdc742bd9c281e9cb2d7768328"
@@ -2691,6 +3076,16 @@
   resolved "https://registry.yarnpkg.com/binary-extensions/-/binary-extensions-2.0.0.tgz#23c0df14f6a88077f5f986c0d167ec03c3d5537c"
   integrity sha512-Phlt0plgpIIBOGTT/ehfFnbNlfsDEiqmzE2KRXoX1bLIlir4X/MR+zSyBEkL05ffWgnRSf/DXv+WrUAVr93/ow==
 
+binary-search-bounds@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/binary-search-bounds/-/binary-search-bounds-1.0.0.tgz#323ca317e3f2a40f4244c7255f5384a5b207bb69"
+  integrity sha1-MjyjF+PypA9CRMclX1OEpbIHu2k=
+
+binary-search-bounds@^2.0.3, binary-search-bounds@^2.0.4:
+  version "2.0.4"
+  resolved "https://registry.yarnpkg.com/binary-search-bounds/-/binary-search-bounds-2.0.4.tgz#eea0e4081da93baa851c7d851a7e636c3d51307f"
+  integrity sha512-2hg5kgdKql5ClF2ErBcSx0U5bnl5hgS4v7wMnLFodyR47yMtj2w+UAZB+0CiqyHct2q543i7Bi4/aMIegorCCg==
+
 bindings@^1.5.0:
   version "1.5.0"
   resolved "https://registry.yarnpkg.com/bindings/-/bindings-1.5.0.tgz#10353c9e945334bc0511a6d90b38fbc7c9c504df"
@@ -2698,6 +3093,31 @@
   dependencies:
     file-uri-to-path "1.0.0"
 
+bit-twiddle@^1.0.0, bit-twiddle@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/bit-twiddle/-/bit-twiddle-1.0.2.tgz#0c6c1fabe2b23d17173d9a61b7b7093eb9e1769e"
+  integrity sha1-DGwfq+KyPRcXPZpht7cJPrnhdp4=
+
+bit-twiddle@~0.0.1:
+  version "0.0.2"
+  resolved "https://registry.yarnpkg.com/bit-twiddle/-/bit-twiddle-0.0.2.tgz#c2eaebb952a3b94acc140497e1cdcd2f1a33f58e"
+  integrity sha1-wurruVKjuUrMFASX4c3NLxoz9Y4=
+
+bitmap-sdf@^1.0.0:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/bitmap-sdf/-/bitmap-sdf-1.0.3.tgz#c99913e5729357a6fd350de34158180c013880b2"
+  integrity sha512-ojYySSvWTx21cbgntR942zgEgqj38wHctN64vr4vYRFf3GKVmI23YlA94meWGkFslidwLwGCsMy2laJ3g/94Sg==
+  dependencies:
+    clamp "^1.0.1"
+
+bl@^1.0.0:
+  version "1.2.2"
+  resolved "https://registry.yarnpkg.com/bl/-/bl-1.2.2.tgz#a160911717103c07410cef63ef51b397c025af9c"
+  integrity sha512-e8tQYnZodmebYDWGH7KMRvtzKXaJHx3BbilrgZCfvyLUYdKpK1t5PSPmpkny/SgiTSCnjfLW7v5rlONXVFkQEA==
+  dependencies:
+    readable-stream "^2.3.5"
+    safe-buffer "^5.1.1"
+
 bluebird@^3.5.5:
   version "3.7.2"
   resolved "https://registry.yarnpkg.com/bluebird/-/bluebird-3.7.2.tgz#9f229c15be272454ffa973ace0dbee79a1b0c36f"
@@ -2708,6 +3128,11 @@
   resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-4.11.8.tgz#2cde09eb5ee341f484746bb0309b3253b1b1442f"
   integrity sha512-ItfYfPLkWHUjckQCk8xC+LwxgK8NYcXywGigJgSwOP8Y2iyWT4f2vsZnoOXTTbo+o5yXmIUJ4gn5538SO5S3gA==
 
+bn.js@^4.11.6:
+  version "4.11.9"
+  resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-4.11.9.tgz#26d556829458f9d1e81fc48952493d0ba3507828"
+  integrity sha512-E6QoYqCKZfgatHTdHzs1RRKP7ip4vvm+EyRUeE2RF0NblwVvb0p6jSVeNTOFxPn26QXN2o6SMfNxKp6kU8zQaw==
+
 body-parser@1.19.0, body-parser@^1.19.0:
   version "1.19.0"
   resolved "https://registry.yarnpkg.com/body-parser/-/body-parser-1.19.0.tgz#96b2709e57c9c4e09a6fd66a8fd979844f69f08a"
@@ -2741,6 +3166,21 @@
   resolved "https://registry.yarnpkg.com/boolbase/-/boolbase-1.0.0.tgz#68dff5fbe60c51eb37725ea9e3ed310dcc1e776e"
   integrity sha1-aN/1++YMUes3cl6p4+0xDcwed24=
 
+boundary-cells@^2.0.0:
+  version "2.0.1"
+  resolved "https://registry.yarnpkg.com/boundary-cells/-/boundary-cells-2.0.1.tgz#e905a8d1419cf47cb36be3dbf525db5e24de0042"
+  integrity sha1-6QWo0UGc9Hyza+Pb9SXbXiTeAEI=
+  dependencies:
+    tape "^4.0.0"
+
+box-intersect@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/box-intersect/-/box-intersect-1.0.2.tgz#4693ad63e828868d0654b114e09364d6281f3fbd"
+  integrity sha512-yJeMwlmFPG1gIa7Rs/cGXeI6iOj6Qz5MG5PE61xLKpElUGzmJ4abm+qsLpzxKJFpsSDq742BQEocr8dI2t8Nxw==
+  dependencies:
+    bit-twiddle "^1.0.2"
+    typedarray-pool "^1.1.0"
+
 boxen@^3.0.0:
   version "3.2.0"
   resolved "https://registry.yarnpkg.com/boxen/-/boxen-3.2.0.tgz#fbdff0de93636ab4450886b6ff45b92d098f45eb"
@@ -2912,6 +3352,27 @@
   dependencies:
     node-int64 "^0.4.0"
 
+buble@^0.19.3:
+  version "0.19.8"
+  resolved "https://registry.yarnpkg.com/buble/-/buble-0.19.8.tgz#d642f0081afab66dccd897d7b6360d94030b9d3d"
+  integrity sha512-IoGZzrUTY5fKXVkgGHw3QeXFMUNBFv+9l8a4QJKG1JhG3nCMHTdEX1DCOg8568E2Q9qvAQIiSokv6Jsgx8p2cA==
+  dependencies:
+    acorn "^6.1.1"
+    acorn-dynamic-import "^4.0.0"
+    acorn-jsx "^5.0.1"
+    chalk "^2.4.2"
+    magic-string "^0.25.3"
+    minimist "^1.2.0"
+    os-homedir "^2.0.0"
+    regexpu-core "^4.5.4"
+
+bubleify@^1.1.0, bubleify@^1.2.0:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/bubleify/-/bubleify-1.2.1.tgz#c11fa33fa59d5b9b747d4e486f43889084257f37"
+  integrity sha512-vp3NHmaQVoKaKWvi15FTMinPNjfp+47+/kFJ9ifezdMF/CBLArCxDVUh+FQE3qRxCRj1qyjJqilTBHHqlM8MaQ==
+  dependencies:
+    buble "^0.19.3"
+
 buf-compare@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/buf-compare/-/buf-compare-1.0.1.tgz#fef28da8b8113a0a0db4430b0b6467b69730b34a"
@@ -3085,6 +3546,11 @@
   resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320"
   integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==
 
+camelcase@^1.0.2:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-1.2.1.tgz#9bb5304d2e0b56698b2c758b08a3eaa9daa58a39"
+  integrity sha1-m7UwTS4LVmmLLHWLCKPqqdqlijk=
+
 camelcase@^4.1.0:
   version "4.1.0"
   resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-4.1.0.tgz#d545635be1e33c542649c69173e5de6acfae34dd"
@@ -3110,6 +3576,13 @@
   resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001054.tgz#7e82fc42d927980b0ce1426c4813df12381e1a75"
   integrity sha512-jiKlTI6Ur8Kjfj8z0muGrV6FscpRvefcQVPSuMuXnvRCfExU7zlVLNjmOz1TnurWgUrAY7MMmjyy+uTgIl1XHw==
 
+canvas-fit@^1.5.0:
+  version "1.5.0"
+  resolved "https://registry.yarnpkg.com/canvas-fit/-/canvas-fit-1.5.0.tgz#ae13be66ade42f5be0e487e345fce30a5e5b5e5f"
+  integrity sha1-rhO+Zq3kL1vg5IfjRfzjCl5bXl8=
+  dependencies:
+    element-size "^1.1.1"
+
 capture-exit@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/capture-exit/-/capture-exit-2.0.0.tgz#fb953bfaebeb781f62898239dabb426d08a509a4"
@@ -3127,6 +3600,28 @@
   resolved "https://registry.yarnpkg.com/caseless/-/caseless-0.12.0.tgz#1b681c21ff84033c826543090689420d187151dc"
   integrity sha1-G2gcIf+EAzyCZUMJBolCDRhxUdw=
 
+cdt2d@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/cdt2d/-/cdt2d-1.0.0.tgz#4f212434bcd67bdb3d68b8fef4acdc2c54415141"
+  integrity sha1-TyEkNLzWe9s9aLj+9KzcLFRBUUE=
+  dependencies:
+    binary-search-bounds "^2.0.3"
+    robust-in-sphere "^1.1.3"
+    robust-orientation "^1.1.3"
+
+cell-orientation@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/cell-orientation/-/cell-orientation-1.0.1.tgz#b504ad96a66ad286d9edd985a2253d03b80d2850"
+  integrity sha1-tQStlqZq0obZ7dmFoiU9A7gNKFA=
+
+center-align@^0.1.1:
+  version "0.1.3"
+  resolved "https://registry.yarnpkg.com/center-align/-/center-align-0.1.3.tgz#aa0d32629b6ee972200411cbd4461c907bc2b7ad"
+  integrity sha1-qg0yYptu6XIgBBHL1EYckHvCt60=
+  dependencies:
+    align-text "^0.1.3"
+    lazy-cache "^1.0.3"
+
 chalk@2.4.2, chalk@^2.0.0, chalk@^2.0.1, chalk@^2.1.0, chalk@^2.4.1, chalk@^2.4.2:
   version "2.4.2"
   resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424"
@@ -3224,6 +3719,26 @@
     inherits "^2.0.1"
     safe-buffer "^5.0.1"
 
+circumcenter@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/circumcenter/-/circumcenter-1.0.0.tgz#20d7aa13b17fbac52f52da4f54c6ac8b906ee529"
+  integrity sha1-INeqE7F/usUvUtpPVMasi5Bu5Sk=
+  dependencies:
+    dup "^1.0.0"
+    robust-linear-solve "^1.0.0"
+
+circumradius@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/circumradius/-/circumradius-1.0.0.tgz#706c447e3e55cd1ed3d11bd133e37c252cc305b5"
+  integrity sha1-cGxEfj5VzR7T0RvRM+N8JSzDBbU=
+  dependencies:
+    circumcenter "^1.0.0"
+
+clamp@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/clamp/-/clamp-1.0.1.tgz#66a0e64011816e37196828fdc8c8c147312c8634"
+  integrity sha1-ZqDmQBGBbjcZaCj9yMjBRzEshjQ=
+
 class-utils@^0.3.5:
   version "0.3.6"
   resolved "https://registry.yarnpkg.com/class-utils/-/class-utils-0.3.6.tgz#f93369ae8b9a7ce02fd41faad0ca83033190c463"
@@ -3246,6 +3761,19 @@
   dependencies:
     source-map "~0.6.0"
 
+clean-pslg@^1.1.0, clean-pslg@^1.1.2:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/clean-pslg/-/clean-pslg-1.1.2.tgz#bd35c7460b7e8ab5a9f761a5ed51796aa3c86c11"
+  integrity sha1-vTXHRgt+irWp92Gl7VF5aqPIbBE=
+  dependencies:
+    big-rat "^1.0.3"
+    box-intersect "^1.0.1"
+    nextafter "^1.0.0"
+    rat-vec "^1.1.1"
+    robust-segment-intersect "^1.0.1"
+    union-find "^1.0.2"
+    uniq "^1.0.1"
+
 clean-regexp@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/clean-regexp/-/clean-regexp-1.0.0.tgz#8df7c7aae51fd36874e8f8d05b9180bc11a3fed7"
@@ -3275,6 +3803,15 @@
   resolved "https://registry.yarnpkg.com/cli-width/-/cli-width-2.2.0.tgz#ff19ede8a9a5e579324147b0c11f0fbcbabed639"
   integrity sha1-/xnt6Kml5XkyQUewwR8PvLq+1jk=
 
+cliui@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/cliui/-/cliui-2.1.0.tgz#4b475760ff80264c762c3a1719032e91c7fea0d1"
+  integrity sha1-S0dXYP+AJkx2LDoXGQMukcf+oNE=
+  dependencies:
+    center-align "^0.1.1"
+    right-align "^0.1.1"
+    wordwrap "0.0.2"
+
 cliui@^4.0.0:
   version "4.1.0"
   resolved "https://registry.yarnpkg.com/cliui/-/cliui-4.1.0.tgz#348422dbe82d800b3022eef4f6ac10bf2e4d1b49"
@@ -3352,6 +3889,13 @@
     map-visit "^1.0.0"
     object-visit "^1.0.0"
 
+color-alpha@^1.0.4:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/color-alpha/-/color-alpha-1.0.4.tgz#c141dc926e95fc3db647d0e14e5bc3651c29e040"
+  integrity sha512-lr8/t5NPozTSqli+duAN+x+no/2WaKTeWvxhHGN+aXT6AJ8vPlzLa7UriyjWak0pSC2jHol9JgjBYnnHsGha9A==
+  dependencies:
+    color-parse "^1.3.8"
+
 color-convert@^1.9.0, color-convert@^1.9.1:
   version "1.9.3"
   resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8"
@@ -3366,6 +3910,13 @@
   dependencies:
     color-name "~1.1.4"
 
+color-id@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/color-id/-/color-id-1.1.0.tgz#5e9159b99a73ac98f74820cb98a15fde3d7e034c"
+  integrity sha512-2iRtAn6dC/6/G7bBIo0uupVrIne1NsQJvJxZOBCzQOfk7jRq97feaDZ3RdzuHakRXXnHGNwglto3pqtRx1sX0g==
+  dependencies:
+    clamp "^1.0.1"
+
 color-name@1.1.3:
   version "1.1.3"
   resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25"
@@ -3376,6 +3927,41 @@
   resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.4.tgz#c2a09a87acbde69543de6f63fa3995c826c536a2"
   integrity sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==
 
+color-normalize@1.5.0, color-normalize@^1.5.0:
+  version "1.5.0"
+  resolved "https://registry.yarnpkg.com/color-normalize/-/color-normalize-1.5.0.tgz#ee610af9acb15daf73e77a945a847b18e40772da"
+  integrity sha512-rUT/HDXMr6RFffrR53oX3HGWkDOP9goSAQGBkUaAYKjOE2JxozccdGyufageWDlInRAjm/jYPrf/Y38oa+7obw==
+  dependencies:
+    clamp "^1.0.1"
+    color-rgba "^2.1.1"
+    dtype "^2.0.0"
+
+color-parse@^1.3.8:
+  version "1.3.8"
+  resolved "https://registry.yarnpkg.com/color-parse/-/color-parse-1.3.8.tgz#eaf54cd385cb34c0681f18c218aca38478082fa3"
+  integrity sha512-1Y79qFv0n1xair3lNMTNeoFvmc3nirMVBij24zbs1f13+7fPpQClMg5b4AuKXLt3szj7BRlHMCXHplkce6XlmA==
+  dependencies:
+    color-name "^1.0.0"
+    defined "^1.0.0"
+    is-plain-obj "^1.1.0"
+
+color-rgba@^2.1.1:
+  version "2.1.1"
+  resolved "https://registry.yarnpkg.com/color-rgba/-/color-rgba-2.1.1.tgz#4633b83817c7406c90b3d7bf4d1acfa48dde5c83"
+  integrity sha512-VaX97wsqrMwLSOR6H7rU1Doa2zyVdmShabKrPEIFywLlHoibgD3QW9Dw6fSqM4+H/LfjprDNAUUW31qEQcGzNw==
+  dependencies:
+    clamp "^1.0.1"
+    color-parse "^1.3.8"
+    color-space "^1.14.6"
+
+color-space@^1.14.6:
+  version "1.16.0"
+  resolved "https://registry.yarnpkg.com/color-space/-/color-space-1.16.0.tgz#611781bca41cd8582a1466fd9e28a7d3d89772a2"
+  integrity sha512-A6WMiFzunQ8KEPFmj02OnnoUnqhmSaHaZ/0LVFcPTdlvm8+3aMJ5x1HRHy3bDHPkovkf4sS0f4wsVvwk71fKkg==
+  dependencies:
+    hsluv "^0.0.3"
+    mumath "^3.3.4"
+
 color-string@^1.5.2:
   version "1.5.3"
   resolved "https://registry.yarnpkg.com/color-string/-/color-string-1.5.3.tgz#c9bbc5f01b58b5492f3d6857459cb6590ce204cc"
@@ -3392,6 +3978,13 @@
     color-convert "^1.9.1"
     color-string "^1.5.2"
 
+colormap@^2.3.1:
+  version "2.3.1"
+  resolved "https://registry.yarnpkg.com/colormap/-/colormap-2.3.1.tgz#9f2ab643591c0728d32332d5480b2487a4e0f249"
+  integrity sha512-TEzNlo/qYp6pBoR2SK9JiV+DG1cmUcVO/+DEJqVPSHIKNlWh5L5L4FYog7b/h0bAnhKhpOAvx/c1dFp2QE9sFw==
+  dependencies:
+    lerp "^1.0.3"
+
 combined-stream@^1.0.6, combined-stream@~1.0.6:
   version "1.0.8"
   resolved "https://registry.yarnpkg.com/combined-stream/-/combined-stream-1.0.8.tgz#c3d45a8b34fd730631a110a8a2520682b31d5a7f"
@@ -3399,7 +3992,7 @@
   dependencies:
     delayed-stream "~1.0.0"
 
-commander@^2.11.0, commander@^2.20.0, commander@~2.20.3:
+commander@2, commander@^2.11.0, commander@^2.15.1, commander@^2.20.0, commander@~2.20.3:
   version "2.20.3"
   resolved "https://registry.yarnpkg.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33"
   integrity sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==
@@ -3419,6 +4012,30 @@
   resolved "https://registry.yarnpkg.com/commondir/-/commondir-1.0.1.tgz#ddd800da0c66127393cca5950ea968a3aaf1253b"
   integrity sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs=
 
+compare-angle@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/compare-angle/-/compare-angle-1.0.1.tgz#a4eb63416ea3c747fc6bd6c8b63668b4de4fa129"
+  integrity sha1-pOtjQW6jx0f8a9bItjZotN5PoSk=
+  dependencies:
+    robust-orientation "^1.0.2"
+    robust-product "^1.0.0"
+    robust-sum "^1.0.0"
+    signum "^0.0.0"
+    two-sum "^1.0.0"
+
+compare-cell@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/compare-cell/-/compare-cell-1.0.0.tgz#a9eb708f6e0e41aef7aa566b130f1968dc9e1aaa"
+  integrity sha1-qetwj24OQa73qlZrEw8ZaNyeGqo=
+
+compare-oriented-cell@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/compare-oriented-cell/-/compare-oriented-cell-1.0.1.tgz#6a149feef9dfc4f8fc62358e51dd42effbbdc39e"
+  integrity sha1-ahSf7vnfxPj8YjWOUd1C7/u9w54=
+  dependencies:
+    cell-orientation "^1.0.1"
+    compare-cell "^1.0.0"
+
 component-classes@1.x, component-classes@^1.2.5, component-classes@^1.2.6:
   version "1.2.6"
   resolved "https://registry.yarnpkg.com/component-classes/-/component-classes-1.2.6.tgz#c642394c3618a4d8b0b8919efccbbd930e5cd691"
@@ -3463,12 +4080,23 @@
     safe-buffer "5.1.2"
     vary "~1.1.2"
 
+compute-dims@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/compute-dims/-/compute-dims-1.1.0.tgz#6d5b712929b6c531af3b4d580ed5adacbbd77e0c"
+  integrity sha512-YHMiIKjH/8Eom8zATk3g8/lH3HxGCZcVQyEfEoVrfWI7od/WRpTgRGShnei3jArYSx77mQqPxZNokjGHCdLfxg==
+  dependencies:
+    utils-copy "^1.0.0"
+    validate.io-array "^1.0.6"
+    validate.io-matrix-like "^1.0.2"
+    validate.io-ndarray-like "^1.0.0"
+    validate.io-positive-integer "^1.0.0"
+
 concat-map@0.0.1:
   version "0.0.1"
   resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b"
   integrity sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=
 
-concat-stream@^1.5.0:
+concat-stream@^1.5.0, concat-stream@^1.5.2, concat-stream@~1.6.0:
   version "1.6.2"
   resolved "https://registry.yarnpkg.com/concat-stream/-/concat-stream-1.6.2.tgz#904bdf194cd3122fc675c77fc4ac3d4ff0fd1a34"
   integrity sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw==
@@ -3478,6 +4106,16 @@
     readable-stream "^2.2.2"
     typedarray "^0.0.6"
 
+concat-stream@~2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/concat-stream/-/concat-stream-2.0.0.tgz#414cf5af790a48c60ab9be4527d56d5e41133cb1"
+  integrity sha512-MWufYdFw53ccGjCA+Ol7XJYpAlW6/prSMzuPOTRnJGcGzuhLn4Scrz7qf6o8bROZ514ltazcIFJZevcfbo0x7A==
+  dependencies:
+    buffer-from "^1.0.0"
+    inherits "^2.0.3"
+    readable-stream "^3.0.2"
+    typedarray "^0.0.6"
+
 configstore@^4.0.0:
   version "4.0.0"
   resolved "https://registry.yarnpkg.com/configstore/-/configstore-4.0.0.tgz#5933311e95d3687efb592c528b922d9262d227e7"
@@ -3522,6 +4160,16 @@
   resolved "https://registry.yarnpkg.com/console-browserify/-/console-browserify-1.2.0.tgz#67063cef57ceb6cf4993a2ab3a55840ae8c49336"
   integrity sha512-ZMkYO/LkF17QvCPqM0gxw8yUzigAOZOSWSHg91FH6orS7vcEj5dVZTidN2fQ14yBSdg97RqhSNwLUXInd52OTA==
 
+const-max-uint32@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/const-max-uint32/-/const-max-uint32-1.0.2.tgz#f009bb6230e678ed874dd2d6a9cd9e3cbfabb676"
+  integrity sha1-8Am7YjDmeO2HTdLWqc2ePL+rtnY=
+
+const-pinf-float64@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/const-pinf-float64/-/const-pinf-float64-1.0.0.tgz#f6efb0d79f9c0986d3e79f2923abf9b70b63d726"
+  integrity sha1-9u+w15+cCYbT558pI6v5twtj1yY=
+
 constants-browserify@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/constants-browserify/-/constants-browserify-1.0.0.tgz#c20b96d8c617748aaf1c16021760cd27fcb8cb75"
@@ -3544,7 +4192,7 @@
   resolved "https://registry.yarnpkg.com/content-type/-/content-type-1.0.4.tgz#e138cc75e040c727b1966fe5e5f8c9aee256fe3b"
   integrity sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA==
 
-convert-source-map@1.7.0, convert-source-map@^1.4.0, convert-source-map@^1.7.0:
+convert-source-map@1.7.0, convert-source-map@^1.4.0, convert-source-map@^1.5.0, convert-source-map@^1.7.0:
   version "1.7.0"
   resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.7.0.tgz#17a2cb882d7f77d3490585e2ce6c524424a3a442"
   integrity sha512-4FJkXzKXEDB1snCFZlLP4gpC3JILicCpGbzG9f9G7tGqGCzETQ2hWPrcinA9oU4wtf2biUaEH5065UnMeR33oA==
@@ -3556,6 +4204,15 @@
   resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-0.3.5.tgz#f1d802950af7dd2631a1febe0596550c86ab3190"
   integrity sha1-8dgClQr33SYxof6+BZZVDIarMZA=
 
+convex-hull@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/convex-hull/-/convex-hull-1.0.3.tgz#20a3aa6ce87f4adea2ff7d17971c9fc1c67e1fff"
+  integrity sha1-IKOqbOh/St6i/30XlxyfwcZ+H/8=
+  dependencies:
+    affine-hull "^1.0.0"
+    incremental-convex-hull "^1.0.1"
+    monotone-convex-hull-2d "^1.0.1"
+
 cookie-signature@1.0.6:
   version "1.0.6"
   resolved "https://registry.yarnpkg.com/cookie-signature/-/cookie-signature-1.0.6.tgz#e303a882b342cc3ee8ca513a79999734dab3ae2c"
@@ -3660,6 +4317,11 @@
     path-type "^4.0.0"
     yaml "^1.7.2"
 
+country-regex@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/country-regex/-/country-regex-1.1.0.tgz#51c333dcdf12927b7e5eeb9c10ac8112a6120896"
+  integrity sha1-UcMz3N8Sknt+XuucEKyBEqYSCJY=
+
 create-ecdh@^4.0.0:
   version "4.0.3"
   resolved "https://registry.yarnpkg.com/create-ecdh/-/create-ecdh-4.0.3.tgz#c9111b6f33045c4697f144787f9254cdc77c45ff"
@@ -3784,6 +4446,46 @@
     postcss "^7.0.1"
     timsort "^0.3.0"
 
+css-font-size-keywords@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/css-font-size-keywords/-/css-font-size-keywords-1.0.0.tgz#854875ace9aca6a8d2ee0d345a44aae9bb6db6cb"
+  integrity sha1-hUh1rOmspqjS7g00WkSq6btttss=
+
+css-font-stretch-keywords@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/css-font-stretch-keywords/-/css-font-stretch-keywords-1.0.1.tgz#50cee9b9ba031fb5c952d4723139f1e107b54b10"
+  integrity sha1-UM7puboDH7XJUtRyMTnx4Qe1SxA=
+
+css-font-style-keywords@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/css-font-style-keywords/-/css-font-style-keywords-1.0.1.tgz#5c3532813f63b4a1de954d13cea86ab4333409e4"
+  integrity sha1-XDUygT9jtKHelU0TzqhqtDM0CeQ=
+
+css-font-weight-keywords@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/css-font-weight-keywords/-/css-font-weight-keywords-1.0.0.tgz#9bc04671ac85bc724b574ef5d3ac96b0d604fd97"
+  integrity sha1-m8BGcayFvHJLV07106yWsNYE/Zc=
+
+css-font@^1.0.0, css-font@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/css-font/-/css-font-1.2.0.tgz#e73cbdc11fd87c8e6c928ad7098a9771c8c2b6e3"
+  integrity sha512-V4U4Wps4dPDACJ4WpgofJ2RT5Yqwe1lEH6wlOOaIxMi0gTjdIijsc5FmxQlZ7ZZyKQkkutqqvULOp07l9c7ssA==
+  dependencies:
+    css-font-size-keywords "^1.0.0"
+    css-font-stretch-keywords "^1.0.1"
+    css-font-style-keywords "^1.0.1"
+    css-font-weight-keywords "^1.0.0"
+    css-global-keywords "^1.0.1"
+    css-system-font-keywords "^1.0.0"
+    pick-by-alias "^1.2.0"
+    string-split-by "^1.0.0"
+    unquote "^1.1.0"
+
+css-global-keywords@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/css-global-keywords/-/css-global-keywords-1.0.1.tgz#72a9aea72796d019b1d2a3252de4e5aaa37e4a69"
+  integrity sha1-cqmupyeW0Bmx0qMlLeTlqqN+Smk=
+
 css-has-pseudo@^0.10.0:
   version "0.10.0"
   resolved "https://registry.yarnpkg.com/css-has-pseudo/-/css-has-pseudo-0.10.0.tgz#3c642ab34ca242c59c41a125df9105841f6966ee"
@@ -3842,6 +4544,11 @@
     domutils "^1.7.0"
     nth-check "^1.0.2"
 
+css-system-font-keywords@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/css-system-font-keywords/-/css-system-font-keywords-1.0.0.tgz#85c6f086aba4eb32c571a3086affc434b84823ed"
+  integrity sha1-hcbwhquk6zLFcaMIav/ENLhII+0=
+
 css-tree@1.0.0-alpha.37:
   version "1.0.0-alpha.37"
   resolved "https://registry.yarnpkg.com/css-tree/-/css-tree-1.0.0-alpha.37.tgz#98bebd62c4c1d9f960ec340cf9f7522e30709a22"
@@ -3875,6 +4582,11 @@
     source-map-resolve "^0.5.2"
     urix "^0.1.0"
 
+csscolorparser@~1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/csscolorparser/-/csscolorparser-1.0.3.tgz#b34f391eea4da8f3e98231e2ccd8df9c041f171b"
+  integrity sha1-s085HupNqPPpgjHizNjfnAQfFxs=
+
 cssdb@^4.4.0:
   version "4.4.0"
   resolved "https://registry.yarnpkg.com/cssdb/-/cssdb-4.4.0.tgz#3bf2f2a68c10f5c6a08abd92378331ee803cddb0"
@@ -3982,6 +4694,16 @@
   resolved "https://registry.yarnpkg.com/csstype/-/csstype-2.6.8.tgz#0fb6fc2417ffd2816a418c9336da74d7f07db431"
   integrity sha512-msVS9qTuMT5zwAGCVm4mxfrZ18BNc6Csd0oJAtiFMZ1FAx1CCvy2+5MDmYoix63LM/6NDbNtodCiGYGmFgO0dA==
 
+csstype@^2.5.7, csstype@^2.6.7:
+  version "2.6.10"
+  resolved "https://registry.yarnpkg.com/csstype/-/csstype-2.6.10.tgz#e63af50e66d7c266edb6b32909cfd0aabe03928b"
+  integrity sha512-D34BqZU4cIlMCY93rZHbrq9pjTAQJ3U8S8rfBqjwHxkGPThWFjzZDQpgMJY0QViLxth6ZKYiwFBo14RdN44U/w==
+
+cubic-hermite@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/cubic-hermite/-/cubic-hermite-1.0.0.tgz#84e3b2f272b31454e8393b99bb6aed45168c14e5"
+  integrity sha1-hOOy8nKzFFToOTuZu2rtRRaMFOU=
+
 currently-unhandled@^0.4.1:
   version "0.4.1"
   resolved "https://registry.yarnpkg.com/currently-unhandled/-/currently-unhandled-0.4.1.tgz#988df33feab191ef799a61369dd76c17adf957ea"
@@ -3996,11 +4718,105 @@
   dependencies:
     lodash.flow "^3.5.0"
 
+cwise-compiler@^1.0.0, cwise-compiler@^1.1.1, cwise-compiler@^1.1.2:
+  version "1.1.3"
+  resolved "https://registry.yarnpkg.com/cwise-compiler/-/cwise-compiler-1.1.3.tgz#f4d667410e850d3a313a7d2db7b1e505bb034cc5"
+  integrity sha1-9NZnQQ6FDToxOn0tt7HlBbsDTMU=
+  dependencies:
+    uniq "^1.0.0"
+
+cwise-parser@^1.0.0:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/cwise-parser/-/cwise-parser-1.0.3.tgz#8e493c17d54f97cb030a9e9854bc86c9dfb354fe"
+  integrity sha1-jkk8F9VPl8sDCp6YVLyGyd+zVP4=
+  dependencies:
+    esprima "^1.0.3"
+    uniq "^1.0.0"
+
+cwise@^1.0.10, cwise@^1.0.4:
+  version "1.0.10"
+  resolved "https://registry.yarnpkg.com/cwise/-/cwise-1.0.10.tgz#24eee6072ebdfd6b8c6f5dadb17090b649b12bef"
+  integrity sha1-JO7mBy69/WuMb12tsXCQtkmxK+8=
+  dependencies:
+    cwise-compiler "^1.1.1"
+    cwise-parser "^1.0.0"
+    static-module "^1.0.0"
+    uglify-js "^2.6.0"
+
 cyclist@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/cyclist/-/cyclist-1.0.1.tgz#596e9698fd0c80e12038c2b82d6eb1b35b6224d9"
   integrity sha1-WW6WmP0MgOEgOMK4LW6xs1tiJNk=
 
+d3-array@1, d3-array@^1.2.1:
+  version "1.2.4"
+  resolved "https://registry.yarnpkg.com/d3-array/-/d3-array-1.2.4.tgz#635ce4d5eea759f6f605863dbcfc30edc737f71f"
+  integrity sha512-KHW6M86R+FUPYGb3R5XiYjXPq7VzwxZ22buHhAEVG5ztoEcZZMLov530mmccaqA1GghZArjQV46fuc8kUqhhHw==
+
+d3-collection@1, d3-collection@^1.0.4:
+  version "1.0.7"
+  resolved "https://registry.yarnpkg.com/d3-collection/-/d3-collection-1.0.7.tgz#349bd2aa9977db071091c13144d5e4f16b5b310e"
+  integrity sha512-ii0/r5f4sjKNTfh84Di+DpztYwqKhEyUlKoPrzUFfeSkWxjW49xU2QzO9qrPrNkpdI0XJkfzvmTu8V2Zylln6A==
+
+d3-color@1:
+  version "1.4.1"
+  resolved "https://registry.yarnpkg.com/d3-color/-/d3-color-1.4.1.tgz#c52002bf8846ada4424d55d97982fef26eb3bc8a"
+  integrity sha512-p2sTHSLCJI2QKunbGb7ocOh7DgTAn8IrLx21QRc/BSnodXM4sv6aLQlnfpvehFMLZEfBc6g9pH9SWQccFYfJ9Q==
+
+d3-dispatch@1:
+  version "1.0.6"
+  resolved "https://registry.yarnpkg.com/d3-dispatch/-/d3-dispatch-1.0.6.tgz#00d37bcee4dd8cd97729dd893a0ac29caaba5d58"
+  integrity sha512-fVjoElzjhCEy+Hbn8KygnmMS7Or0a9sI2UzGwoB7cCtvI1XpVN9GpoYlnb3xt2YV66oXYb1fLJ8GMvP4hdU1RA==
+
+d3-force@^1.2.1:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/d3-force/-/d3-force-1.2.1.tgz#fd29a5d1ff181c9e7f0669e4bd72bdb0e914ec0b"
+  integrity sha512-HHvehyaiUlVo5CxBJ0yF/xny4xoaxFxDnBXNvNcfW9adORGZfyNF1dj6DGLKyk4Yh3brP/1h3rnDzdIAwL08zg==
+  dependencies:
+    d3-collection "1"
+    d3-dispatch "1"
+    d3-quadtree "1"
+    d3-timer "1"
+
+d3-hierarchy@^1.1.9:
+  version "1.1.9"
+  resolved "https://registry.yarnpkg.com/d3-hierarchy/-/d3-hierarchy-1.1.9.tgz#2f6bee24caaea43f8dc37545fa01628559647a83"
+  integrity sha512-j8tPxlqh1srJHAtxfvOUwKNYJkQuBFdM1+JAUfq6xqH5eAqf93L7oG1NVqDa4CpFZNvnNKtCYEUC8KY9yEn9lQ==
+
+d3-interpolate@^1.4.0:
+  version "1.4.0"
+  resolved "https://registry.yarnpkg.com/d3-interpolate/-/d3-interpolate-1.4.0.tgz#526e79e2d80daa383f9e0c1c1c7dcc0f0583e987"
+  integrity sha512-V9znK0zc3jOPV4VD2zZn0sDhZU3WAE2bmlxdIwwQPPzPjvyLkd8B3JUVdS1IDUFDkWZ72c9qnv1GK2ZagTZ8EA==
+  dependencies:
+    d3-color "1"
+
+d3-path@1:
+  version "1.0.9"
+  resolved "https://registry.yarnpkg.com/d3-path/-/d3-path-1.0.9.tgz#48c050bb1fe8c262493a8caf5524e3e9591701cf"
+  integrity sha512-VLaYcn81dtHVTjEHd8B+pbe9yHWpXKZUC87PzoFmsFrJqgFwDe/qxfp5MlfsfM1V5E/iVt0MmEbWQ7FVIXh/bg==
+
+d3-quadtree@1:
+  version "1.0.7"
+  resolved "https://registry.yarnpkg.com/d3-quadtree/-/d3-quadtree-1.0.7.tgz#ca8b84df7bb53763fe3c2f24bd435137f4e53135"
+  integrity sha512-RKPAeXnkC59IDGD0Wu5mANy0Q2V28L+fNe65pOCXVdVuTJS3WPKaJlFHer32Rbh9gIo9qMuJXio8ra4+YmIymA==
+
+d3-shape@^1.2.0:
+  version "1.3.7"
+  resolved "https://registry.yarnpkg.com/d3-shape/-/d3-shape-1.3.7.tgz#df63801be07bc986bc54f63789b4fe502992b5d7"
+  integrity sha512-EUkvKjqPFUAZyOlhY5gzCxCeI0Aep04LwIRpsZ/mLFelJiUfnK56jo5JMDSE7yyP2kLSb6LtF+S5chMk7uqPqw==
+  dependencies:
+    d3-path "1"
+
+d3-timer@1:
+  version "1.0.10"
+  resolved "https://registry.yarnpkg.com/d3-timer/-/d3-timer-1.0.10.tgz#dfe76b8a91748831b13b6d9c793ffbd508dd9de5"
+  integrity sha512-B1JDm0XDaQC+uvo4DT79H0XmBskgS3l6Ve+1SBCfxgmtIb1AVrPIoqd+nPSv+loMX8szQ0sVUhGngL7D5QPiXw==
+
+d3@^3.5.17:
+  version "3.5.17"
+  resolved "https://registry.yarnpkg.com/d3/-/d3-3.5.17.tgz#bc46748004378b21a360c9fc7cf5231790762fb8"
+  integrity sha1-vEZ0gAQ3iyGjYMn8fPUjF5B2L7g=
+
 d@1, d@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/d/-/d-1.0.1.tgz#8698095372d58dbee346ffd0c7093f99f8f9eb5a"
@@ -4066,7 +4882,7 @@
     decamelize "^1.1.0"
     map-obj "^1.0.0"
 
-decamelize@^1.1.0, decamelize@^1.2.0:
+decamelize@^1.0.0, decamelize@^1.1.0, decamelize@^1.2.0:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290"
   integrity sha1-9lNNFRSCabIDUue+4m9QH5oZEpA=
@@ -4083,7 +4899,7 @@
   dependencies:
     mimic-response "^1.0.0"
 
-deep-equal@^1.0.1:
+deep-equal@^1.0.1, deep-equal@~1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/deep-equal/-/deep-equal-1.1.1.tgz#b5c98c942ceffaf7cb051e24e1434a25a2e6076a"
   integrity sha512-yd9c5AdiqVcR+JjcwUQb9DkhJc8ngNr0MahEBGvDiJw8puWab2yZlh+nkasOnZP+EGTAP6rRp2JzJhJZzvNF8g==
@@ -4154,6 +4970,11 @@
     is-descriptor "^1.0.2"
     isobject "^3.0.1"
 
+defined@^1.0.0, defined@~1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/defined/-/defined-1.0.0.tgz#c98d9bcef75674188e110969151199e39b1fa693"
+  integrity sha1-yY2bzvdWdBiOEQlpFRGZ45sfppM=
+
 del@^4.1.1:
   version "4.1.1"
   resolved "https://registry.yarnpkg.com/del/-/del-4.1.1.tgz#9e8f117222ea44a31ff3a156c049b99052a9f0b4"
@@ -4167,6 +4988,14 @@
     pify "^4.0.1"
     rimraf "^2.6.3"
 
+delaunay-triangulate@^1.1.6:
+  version "1.1.6"
+  resolved "https://registry.yarnpkg.com/delaunay-triangulate/-/delaunay-triangulate-1.1.6.tgz#5bbca21b078198d4bc3c75796a35cbb98c25954c"
+  integrity sha1-W7yiGweBmNS8PHV5ajXLuYwllUw=
+  dependencies:
+    incremental-convex-hull "^1.0.1"
+    uniq "^1.0.1"
+
 delayed-stream@~1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/delayed-stream/-/delayed-stream-1.0.0.tgz#df3ae199acadfb7d440aaae0b29e2272b24ec619"
@@ -4190,6 +5019,11 @@
   resolved "https://registry.yarnpkg.com/destroy/-/destroy-1.0.4.tgz#978857442c44749e4206613e37946205826abd80"
   integrity sha1-l4hXRCxEdJ5CBmE+N5RiBYJqvYA=
 
+detect-kerning@^2.1.2:
+  version "2.1.2"
+  resolved "https://registry.yarnpkg.com/detect-kerning/-/detect-kerning-2.1.2.tgz#4ecd548e4a5a3fc880fe2a50609312d000fa9fc2"
+  integrity sha512-I3JIbrnKPAntNLl1I6TpSQQdQ4AutYzv/sKMFKbepawV/hlH0GmYKhUoOEMd4xqaUHT+Bm0f4127lh5qs1m1tw==
+
 detect-newline@^2.1.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/detect-newline/-/detect-newline-2.1.0.tgz#f41f1c10be4b00e87b5f13da680759f2c5bfd3e2"
@@ -4298,6 +5132,14 @@
   dependencies:
     utila "~0.4"
 
+dom-helpers@^5.0.1:
+  version "5.1.4"
+  resolved "https://registry.yarnpkg.com/dom-helpers/-/dom-helpers-5.1.4.tgz#4609680ab5c79a45f2531441f1949b79d6587f4b"
+  integrity sha512-TjMyeVUvNEnOnhzs6uAn9Ya47GmMo3qq7m+Lr/3ON0Rs5kHvb8I+SQYjLUSYn7qhEm0QjW0yrBkvz9yOrwwz1A==
+  dependencies:
+    "@babel/runtime" "^7.8.7"
+    csstype "^2.6.7"
+
 dom-matches@>=1.0.1:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/dom-matches/-/dom-matches-2.0.0.tgz#d2728b416a87533980eb089b848d253cf23a758c"
@@ -4393,6 +5235,18 @@
   resolved "https://registry.yarnpkg.com/dotenv/-/dotenv-8.2.0.tgz#97e619259ada750eea3e4ea3e26bceea5424b16a"
   integrity sha512-8sJ78ElpbDJBHNeBzUbUVLsqKdccaa/BXF1uPTw3GrvQTBgrQrtObr2mUrE38vzYd8cEv+m/JBfDLioYcfXoaw==
 
+dotignore@~0.1.2:
+  version "0.1.2"
+  resolved "https://registry.yarnpkg.com/dotignore/-/dotignore-0.1.2.tgz#f942f2200d28c3a76fbdd6f0ee9f3257c8a2e905"
+  integrity sha512-UGGGWfSauusaVJC+8fgV+NVvBXkCTmVv7sk6nojDZZvuOUNGUy0Zk4UpHQD6EDjS0jpBwcACvH4eofvyzBcRDw==
+  dependencies:
+    minimatch "^3.0.4"
+
+double-bits@^1.1.0, double-bits@^1.1.1:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/double-bits/-/double-bits-1.1.1.tgz#58abba45494da4d0fa36b73ad11a286c9184b1c6"
+  integrity sha1-WKu6RUlNpND6Nrc60RoobJGEscY=
+
 draft-js@^0.10.0, draft-js@~0.10.0:
   version "0.10.5"
   resolved "https://registry.yarnpkg.com/draft-js/-/draft-js-0.10.5.tgz#bfa9beb018fe0533dbb08d6675c371a6b08fa742"
@@ -4402,6 +5256,31 @@
     immutable "~3.7.4"
     object-assign "^4.1.0"
 
+draw-svg-path@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/draw-svg-path/-/draw-svg-path-1.0.0.tgz#6f116d962dd314b99ea534d6f58dd66cdbd69379"
+  integrity sha1-bxFtli3TFLmepTTW9Y3WbNvWk3k=
+  dependencies:
+    abs-svg-path "~0.1.1"
+    normalize-svg-path "~0.1.0"
+
+dtype@^2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/dtype/-/dtype-2.0.0.tgz#cd052323ce061444ecd2e8f5748f69a29be28434"
+  integrity sha1-zQUjI84GFETs0uj1dI9popvihDQ=
+
+dup@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/dup/-/dup-1.0.0.tgz#51fc5ac685f8196469df0b905e934b20af5b4029"
+  integrity sha1-UfxaxoX4GWRp3wuQXpNLIK9bQCk=
+
+duplexer2@~0.0.2:
+  version "0.0.2"
+  resolved "https://registry.yarnpkg.com/duplexer2/-/duplexer2-0.0.2.tgz#c614dcf67e2fb14995a91711e5a617e8a60a31db"
+  integrity sha1-xhTc9n4vsUmVqRcR5aYX6KYKMds=
+  dependencies:
+    readable-stream "~1.1.9"
+
 duplexer3@^0.1.4:
   version "0.1.4"
   resolved "https://registry.yarnpkg.com/duplexer3/-/duplexer3-0.1.4.tgz#ee01dd1cac0ed3cbc7fdbea37dc0a8f1ce002ce2"
@@ -4412,7 +5291,7 @@
   resolved "https://registry.yarnpkg.com/duplexer/-/duplexer-0.1.1.tgz#ace6ff808c1ce66b57d1ebf97977acb02334cfc1"
   integrity sha1-rOb/gIwc5mtX0ev5eXessCM0z8E=
 
-duplexify@^3.4.2, duplexify@^3.6.0:
+duplexify@^3.4.2, duplexify@^3.4.5, duplexify@^3.6.0:
   version "3.7.1"
   resolved "https://registry.yarnpkg.com/duplexify/-/duplexify-3.7.1.tgz#2a4df5317f6ccfd91f86d6fd25d8d8a103b88309"
   integrity sha512-07z8uv2wMyS51kKhD1KsdXJg5WQ6t93RneqRxUHnskXVtlYYkLqM0gqStQZ3pj073g687jPCHrqNfCzawLYh5g==
@@ -4422,6 +5301,11 @@
     readable-stream "^2.0.0"
     stream-shift "^1.0.0"
 
+earcut@^2.1.5, earcut@^2.2.2:
+  version "2.2.2"
+  resolved "https://registry.yarnpkg.com/earcut/-/earcut-2.2.2.tgz#41b0bc35f63e0fe80da7cddff28511e7e2e80d11"
+  integrity sha512-eZoZPPJcUHnfRZ0PjLvx2qBordSiO8ofC3vt+qACLM95u+4DovnbYNpQtJh0DNsWj8RnxrQytD4WA8gj5cRIaQ==
+
 ecc-jsbn@~0.1.1:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz#3a83a904e54353287874c564b7549386849a98c9"
@@ -4430,6 +5314,13 @@
     jsbn "~0.1.0"
     safer-buffer "^2.1.0"
 
+edges-to-adjacency-list@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/edges-to-adjacency-list/-/edges-to-adjacency-list-1.0.0.tgz#c146d2e084addfba74a51293c6e0199a49f757f1"
+  integrity sha1-wUbS4ISt37p0pRKTxuAZmkn3V/E=
+  dependencies:
+    uniq "^1.0.0"
+
 ee-first@1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d"
@@ -4445,6 +5336,18 @@
   resolved "https://registry.yarnpkg.com/electron-to-chromium/-/electron-to-chromium-1.3.431.tgz#705dd8ef46200415ba837b31d927cdc1e43db303"
   integrity sha512-2okqkXCIda7qDwjYhUFxPcQdZDIZZ/zBLDzVOif7WW/TSNfEhdT6SO07O1x/sFteEHX189Z//UwjbZKKCOn2Fg==
 
+element-size@^1.1.1:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/element-size/-/element-size-1.1.1.tgz#64e5f159d97121631845bcbaecaf279c39b5e34e"
+  integrity sha1-ZOXxWdlxIWMYRby67K8nnDm1404=
+
+elementary-circuits-directed-graph@^1.0.4:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/elementary-circuits-directed-graph/-/elementary-circuits-directed-graph-1.2.0.tgz#c6507f42566507c646b5bb144b892fbce1656371"
+  integrity sha512-eOQofnrNqebPtC29PvyNMGUBdMrIw5i8nOoC/2VOlSF84tf5+ZXnRkIk7TgdT22jFXK68CC7aA881KRmNYf/Pg==
+  dependencies:
+    strongly-connected-components "^1.0.1"
+
 elliptic@^6.0.0:
   version "6.5.2"
   resolved "https://registry.yarnpkg.com/elliptic/-/elliptic-6.5.2.tgz#05c5678d7173c049d8ca433552224a495d0e3762"
@@ -4590,7 +5493,7 @@
     is-date-object "^1.0.1"
     is-symbol "^1.0.2"
 
-es5-ext@^0.10.35, es5-ext@^0.10.50:
+es5-ext@^0.10.35, es5-ext@^0.10.46, es5-ext@^0.10.50:
   version "0.10.53"
   resolved "https://registry.yarnpkg.com/es5-ext/-/es5-ext-0.10.53.tgz#93c5a3acfdbef275220ad72644ad02ee18368de1"
   integrity sha512-Xs2Stw6NiNHWypzRTY1MtaG/uJlwCk8kH81920ma8mvN8Xq1gsfhZvpkImLQArw8AHnv8MT2I45J3c0R8slE+Q==
@@ -4599,7 +5502,7 @@
     es6-symbol "~3.1.3"
     next-tick "~1.0.0"
 
-es6-iterator@2.0.3, es6-iterator@~2.0.3:
+es6-iterator@2.0.3, es6-iterator@^2.0.3, es6-iterator@~2.0.3:
   version "2.0.3"
   resolved "https://registry.yarnpkg.com/es6-iterator/-/es6-iterator-2.0.3.tgz#a7de889141a05a94b0854403b2d0a0fbfa98f3b7"
   integrity sha1-p96IkUGgWpSwhUQDstCg+/qY87c=
@@ -4608,6 +5511,11 @@
     es5-ext "^0.10.35"
     es6-symbol "^3.1.1"
 
+es6-promise@^4.2.8:
+  version "4.2.8"
+  resolved "https://registry.yarnpkg.com/es6-promise/-/es6-promise-4.2.8.tgz#4eb21594c972bc40553d276e510539143db53e0a"
+  integrity sha512-HJDGx5daxeIvxdBxvG2cb9g4tEvwIk3i8+nhX0yGrYmZUzbkdg8QbDevheDB8gd0//uPj4c1EQua8Q+MViT0/w==
+
 es6-symbol@^3.1.1, es6-symbol@~3.1.3:
   version "3.1.3"
   resolved "https://registry.yarnpkg.com/es6-symbol/-/es6-symbol-3.1.3.tgz#bad5d3c1bcdac28269f4cb331e431c78ac705d18"
@@ -4616,6 +5524,16 @@
     d "^1.0.1"
     ext "^1.1.2"
 
+es6-weak-map@^2.0.3:
+  version "2.0.3"
+  resolved "https://registry.yarnpkg.com/es6-weak-map/-/es6-weak-map-2.0.3.tgz#b6da1f16cc2cc0d9be43e6bdbfc5e7dfcdf31d53"
+  integrity sha512-p5um32HOTO1kP+w7PRnB+5lQ43Z6muuMuIMffvDN8ZB4GcnjLBV6zGStpbASIMk4DCAvEaamhe2zhyCb/QXXsA==
+  dependencies:
+    d "1"
+    es5-ext "^0.10.46"
+    es6-iterator "^2.0.3"
+    es6-symbol "^3.1.1"
+
 escape-goat@^2.0.0:
   version "2.1.1"
   resolved "https://registry.yarnpkg.com/escape-goat/-/escape-goat-2.1.1.tgz#1b2dc77003676c457ec760b2dc68edb648188675"
@@ -4648,6 +5566,39 @@
   optionalDependencies:
     source-map "~0.6.1"
 
+escodegen@^1.11.1:
+  version "1.14.1"
+  resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-1.14.1.tgz#ba01d0c8278b5e95a9a45350142026659027a457"
+  integrity sha512-Bmt7NcRySdIfNPfU2ZoXDrrXsG9ZjvDxcAlMfDUgRBjLOWTuIACXPBFJH7Z+cLb40JeQco5toikyc9t9P8E9SQ==
+  dependencies:
+    esprima "^4.0.1"
+    estraverse "^4.2.0"
+    esutils "^2.0.2"
+    optionator "^0.8.1"
+  optionalDependencies:
+    source-map "~0.6.1"
+
+escodegen@~0.0.24:
+  version "0.0.28"
+  resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-0.0.28.tgz#0e4ff1715f328775d6cab51ac44a406cd7abffd3"
+  integrity sha1-Dk/xcV8yh3XWyrUaxEpAbNer/9M=
+  dependencies:
+    esprima "~1.0.2"
+    estraverse "~1.3.0"
+  optionalDependencies:
+    source-map ">= 0.1.2"
+
+escodegen@~1.3.2:
+  version "1.3.3"
+  resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-1.3.3.tgz#f024016f5a88e046fd12005055e939802e6c5f23"
+  integrity sha1-8CQBb1qI4Eb9EgBQVek5gC5sXyM=
+  dependencies:
+    esprima "~1.1.1"
+    estraverse "~1.5.0"
+    esutils "~1.0.0"
+  optionalDependencies:
+    source-map "~0.1.33"
+
 eslint-ast-utils@^1.1.0:
   version "1.1.0"
   resolved "https://registry.yarnpkg.com/eslint-ast-utils/-/eslint-ast-utils-1.1.0.tgz#3d58ba557801cfb1c941d68131ee9f8c34bd1586"
@@ -5007,16 +5958,31 @@
     acorn-jsx "^5.2.0"
     eslint-visitor-keys "^1.1.0"
 
+esprima@^1.0.3:
+  version "1.2.5"
+  resolved "https://registry.yarnpkg.com/esprima/-/esprima-1.2.5.tgz#0993502feaf668138325756f30f9a51feeec11e9"
+  integrity sha1-CZNQL+r2aBODJXVvMPmlH+7sEek=
+
 esprima@^3.1.3:
   version "3.1.3"
   resolved "https://registry.yarnpkg.com/esprima/-/esprima-3.1.3.tgz#fdca51cee6133895e3c88d535ce49dbff62a4633"
   integrity sha1-/cpRzuYTOJXjyI1TXOSdv/YqRjM=
 
-esprima@^4.0.0:
+esprima@^4.0.0, esprima@^4.0.1:
   version "4.0.1"
   resolved "https://registry.yarnpkg.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71"
   integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A==
 
+esprima@~1.0.2:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/esprima/-/esprima-1.0.4.tgz#9f557e08fc3b4d26ece9dd34f8fbf476b62585ad"
+  integrity sha1-n1V+CPw7TSbs6d00+Pv0drYlha0=
+
+esprima@~1.1.1:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/esprima/-/esprima-1.1.1.tgz#5b6f1547f4d102e670e140c509be6771d6aeb549"
+  integrity sha1-W28VR/TRAuZw4UDFCb5ncdautUk=
+
 espurify@^2.0.1:
   version "2.0.1"
   resolved "https://registry.yarnpkg.com/espurify/-/espurify-2.0.1.tgz#c25b3bb613863daa142edcca052370a1a459f41d"
@@ -5041,11 +6007,26 @@
   resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d"
   integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw==
 
+estraverse@~1.3.0:
+  version "1.3.2"
+  resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-1.3.2.tgz#37c2b893ef13d723f276d878d60d8535152a6c42"
+  integrity sha1-N8K4k+8T1yPydth41g2FNRUqbEI=
+
+estraverse@~1.5.0:
+  version "1.5.1"
+  resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-1.5.1.tgz#867a3e8e58a9f84618afb6c2ddbcd916b7cbaf71"
+  integrity sha1-hno+jlip+EYYr7bC3bzZFrfLr3E=
+
 esutils@^2.0.0, esutils@^2.0.2:
   version "2.0.3"
   resolved "https://registry.yarnpkg.com/esutils/-/esutils-2.0.3.tgz#74d2eb4de0b8da1293711910d50775b9b710ef64"
   integrity sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g==
 
+esutils@~1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/esutils/-/esutils-1.0.0.tgz#8151d358e20c8acc7fb745e7472c0025fe496570"
+  integrity sha1-gVHTWOIMisx/t0XnRywAJf5JZXA=
+
 etag@~1.8.1:
   version "1.8.1"
   resolved "https://registry.yarnpkg.com/etag/-/etag-1.8.1.tgz#41ae2eeb65efa62268aebfea83ac7d79299b0887"
@@ -5061,6 +6042,11 @@
   resolved "https://registry.yarnpkg.com/eventlistener/-/eventlistener-0.0.1.tgz#ed2baabb852227af2bcf889152c72c63ca532eb8"
   integrity sha1-7Suqu4UiJ68rz4iRUscsY8pTLrg=
 
+events@^1.0.2:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/events/-/events-1.1.1.tgz#9ebdb7635ad099c70dcc4c2a1f5004288e8bd924"
+  integrity sha1-nr23Y1rQmccNzEwqH1AEKI6L2SQ=
+
 events@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/events/-/events-3.0.0.tgz#9a0a0dfaf62893d92b875b8f2698ca4114973e88"
@@ -5236,6 +6222,11 @@
     snapdragon "^0.8.1"
     to-regex "^3.0.1"
 
+extract-frustum-planes@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/extract-frustum-planes/-/extract-frustum-planes-1.0.0.tgz#97d5703ff0564c8c3c6838cac45f9e7bc52c9ef5"
+  integrity sha1-l9VwP/BWTIw8aDjKxF+ee8UsnvU=
+
 extsprintf@1.3.0:
   version "1.3.0"
   resolved "https://registry.yarnpkg.com/extsprintf/-/extsprintf-1.3.0.tgz#96918440e3041a7a414f8c52e3c574eb3c3e1e05"
@@ -5246,6 +6237,16 @@
   resolved "https://registry.yarnpkg.com/extsprintf/-/extsprintf-1.4.0.tgz#e2689f8f356fad62cca65a3a91c5df5f9551692f"
   integrity sha1-4mifjzVvrWLMplo6kcXfX5VRaS8=
 
+falafel@^2.1.0:
+  version "2.2.4"
+  resolved "https://registry.yarnpkg.com/falafel/-/falafel-2.2.4.tgz#b5d86c060c2412a43166243cb1bce44d1abd2819"
+  integrity sha512-0HXjo8XASWRmsS0X1EkhwEMZaD3Qvp7FfURwjLKjG1ghfRm/MGZl2r4cWUTv41KdNghTw4OUMmVtdGQp3+H+uQ==
+  dependencies:
+    acorn "^7.1.1"
+    foreach "^2.0.5"
+    isarray "^2.0.1"
+    object-keys "^1.0.6"
+
 fast-deep-equal@^2.0.1:
   version "2.0.1"
   resolved "https://registry.yarnpkg.com/fast-deep-equal/-/fast-deep-equal-2.0.1.tgz#7b05218ddf9667bf7f370bf7fdb2cb15fdd0aa49"
@@ -5273,6 +6274,13 @@
     merge2 "^1.2.3"
     micromatch "^3.1.10"
 
+fast-isnumeric@^1.1.4:
+  version "1.1.4"
+  resolved "https://registry.yarnpkg.com/fast-isnumeric/-/fast-isnumeric-1.1.4.tgz#e165786ff471c439e9ace2b8c8e66cceb47e2ea4"
+  integrity sha512-1mM8qOr2LYz8zGaUdmiqRDiuue00Dxjgcb1NQR7TnhLVh6sQyngP9xvLo7Sl7LZpP/sk5eb+bcyWXw530NTBZw==
+  dependencies:
+    is-string-blank "^1.0.1"
+
 fast-json-stable-stringify@^2.0.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz#874bf69c6f404c2b5d99c481341399fd55892633"
@@ -5349,11 +6357,16 @@
   resolved "https://registry.yarnpkg.com/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz#553a7b8446ff6f684359c445f1e37a05dacc33dd"
   integrity sha512-0Zt+s3L7Vf1biwWZ29aARiVYLx7iMGnEUl9x33fbB/j3jR81u/O2LbqK+Bm1CDSNDKVtJ/YjwY7TUd5SkeLQLw==
 
-filesize@6.0.1, filesize@^6.0.1:
+filesize@6.0.1:
   version "6.0.1"
   resolved "https://registry.yarnpkg.com/filesize/-/filesize-6.0.1.tgz#f850b509909c7c86f7e450ea19006c31c2ed3d2f"
   integrity sha512-u4AYWPgbI5GBhs6id1KdImZWn5yfyFrrQ8OWZdN7ZMfA8Bf4HcO0BGo9bmUIEV8yrp8I1xVfJ/dn90GtFNNJcg==
 
+filesize@^6.1.0:
+  version "6.1.0"
+  resolved "https://registry.yarnpkg.com/filesize/-/filesize-6.1.0.tgz#e81bdaa780e2451d714d71c0d7a4f3238d37ad00"
+  integrity sha512-LpCHtPQ3sFx67z+uh2HnSyWSLLu5Jxo21795uRDuar/EOuYWXib5EmPaGIBuSnRqH2IODiKA2k5re/K9OnN/Yg==
+
 fill-range@^4.0.0:
   version "4.0.0"
   resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-4.0.0.tgz#d544811d428f98eb06a63dc402d2403c328c38f7"
@@ -5371,6 +6384,14 @@
   dependencies:
     to-regex-range "^5.0.1"
 
+filtered-vector@^1.2.1:
+  version "1.2.4"
+  resolved "https://registry.yarnpkg.com/filtered-vector/-/filtered-vector-1.2.4.tgz#56453c034df4302d293ca8ecdeac3f90abc678d3"
+  integrity sha1-VkU8A030MC0pPKjs3qw/kKvGeNM=
+  dependencies:
+    binary-search-bounds "^1.0.0"
+    cubic-hermite "^1.0.0"
+
 finalhandler@~1.1.2:
   version "1.1.2"
   resolved "https://registry.yarnpkg.com/finalhandler/-/finalhandler-1.1.2.tgz#b7e7d000ffd11938d0fdb053506f6ebabe9f587d"
@@ -5460,11 +6481,23 @@
   resolved "https://registry.yarnpkg.com/flatted/-/flatted-2.0.1.tgz#69e57caa8f0eacbc281d2e2cb458d46fdb449e08"
   integrity sha512-a1hQMktqW9Nmqr5aktAux3JMNqaucxGcjtjWnZLHX7yyPCmlSV3M54nGYbqT8K+0GhF3NBgmJCc3ma+WOgX8Jg==
 
+flatten-vertex-data@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/flatten-vertex-data/-/flatten-vertex-data-1.0.2.tgz#889fd60bea506006ca33955ee1105175fb620219"
+  integrity sha512-BvCBFK2NZqerFTdMDgqfHBwxYWnxeCkwONsw6PvBMcUXqo8U/KDWwmXhqx1x2kLIg7DqIsJfOaJFOmlua3Lxuw==
+  dependencies:
+    dtype "^2.0.0"
+
 flatten@^1.0.2:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/flatten/-/flatten-1.0.3.tgz#c1283ac9f27b368abc1e36d1ff7b04501a30356b"
   integrity sha512-dVsPA/UwQ8+2uoFe5GHtiBMu48dWLTdsuEd7CKGlZlD78r1TTWBvDuFaFGKCo/ZfEr95Uk56vZoX86OsHkUeIg==
 
+flip-pixels@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/flip-pixels/-/flip-pixels-1.0.2.tgz#aad7b7d9fc65932d5f27e2e4dac4b494140845e4"
+  integrity sha512-oXbJGbjDnfJRWPC7Va38EFhd+A8JWE5/hCiKcK8qjCdbLj9DTpsq6MEudwpRTH+V4qq+Jw7d3pUgQdSr3x3mTA==
+
 flush-write-stream@^1.0.0:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/flush-write-stream/-/flush-write-stream-1.1.1.tgz#8dd7d873a1babc207d94ead0c2e0e44276ebf2e8"
@@ -5487,6 +6520,27 @@
   dependencies:
     debug "^3.0.0"
 
+font-atlas@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/font-atlas/-/font-atlas-2.1.0.tgz#aa2d6dcf656a6c871d66abbd3dfbea2f77178348"
+  integrity sha512-kP3AmvX+HJpW4w3d+PiPR2X6E1yvsBXt2yhuCw+yReO9F1WYhvZwx3c95DGZGwg9xYzDGrgJYa885xmVA+28Cg==
+  dependencies:
+    css-font "^1.0.0"
+
+font-measure@^1.2.2:
+  version "1.2.2"
+  resolved "https://registry.yarnpkg.com/font-measure/-/font-measure-1.2.2.tgz#41dbdac5d230dbf4db08865f54da28a475e83026"
+  integrity sha512-mRLEpdrWzKe9hbfaF3Qpr06TAjquuBVP5cHy4b3hyeNdjc9i0PO6HniGsX5vjL5OWv7+Bd++NiooNpT/s8BvIA==
+  dependencies:
+    css-font "^1.2.0"
+
+for-each@~0.3.3:
+  version "0.3.3"
+  resolved "https://registry.yarnpkg.com/for-each/-/for-each-0.3.3.tgz#69b447e88a0a5d32c3e7084f3f1710034b21376e"
+  integrity sha512-jqYfLp7mo9vIyQf8ykW2v7A+2N4QjeCeI5+Dz9XraiO1ign81wjiH7Fb9vSOWvQfNtmSa4H2RoQTrrXivdUZmw==
+  dependencies:
+    is-callable "^1.1.3"
+
 for-in@^0.1.3:
   version "0.1.8"
   resolved "https://registry.yarnpkg.com/for-in/-/for-in-0.1.8.tgz#d8773908e31256109952b1fdb9b3fa867d2775e1"
@@ -5504,6 +6558,11 @@
   dependencies:
     for-in "^1.0.1"
 
+foreach@^2.0.5:
+  version "2.0.5"
+  resolved "https://registry.yarnpkg.com/foreach/-/foreach-2.0.5.tgz#0bee005018aeb260d0a3af3ae658dd0136ec1b99"
+  integrity sha1-C+4AUBiusmDQo6865ljdATbsG5k=
+
 forever-agent@~0.6.1:
   version "0.6.1"
   resolved "https://registry.yarnpkg.com/forever-agent/-/forever-agent-0.6.1.tgz#fbc71f0c41adeb37f96c577ad1ed42d8fdacca91"
@@ -5549,7 +6608,7 @@
   resolved "https://registry.yarnpkg.com/fresh/-/fresh-0.5.2.tgz#3d8cadd90d976569fa835ab1f8e4b23a105605a7"
   integrity sha1-PYyt2Q2XZWn6g1qx+OSyOhBWBac=
 
-from2@^2.1.0:
+from2@^2.1.0, from2@^2.3.0:
   version "2.3.0"
   resolved "https://registry.yarnpkg.com/from2/-/from2-2.3.0.tgz#8bfb5502bde4a4d36cfdeea007fcca21d7e382af"
   integrity sha1-i/tVAr3kpNNs/e6gB/zKIdfjgq8=
@@ -5634,21 +6693,31 @@
   resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.1.3.tgz#fb738703ae8d2f9fe900c33836ddebee8b97f23e"
   integrity sha512-Auw9a4AxqWpa9GUfj370BMPzzyncfBABW8Mab7BGWBYDj4Isgq+cDKtx0i6u9jcX9pQDnswsaaOTgTmA5pEjuQ==
 
-function-bind@^1.1.1:
+function-bind@^1.1.1, function-bind@~1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d"
   integrity sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==
 
-functional-red-black-tree@^1.0.1:
+functional-red-black-tree@^1.0.0, functional-red-black-tree@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz#1b0ab3bd553b2a0d6399d29c0e3ea0b252078327"
   integrity sha1-GwqzvVU7Kg1jmdKcDj6gslIHgyc=
 
+gamma@^0.1.0:
+  version "0.1.0"
+  resolved "https://registry.yarnpkg.com/gamma/-/gamma-0.1.0.tgz#3315643403bf27906ca80ab37c36ece9440ef330"
+  integrity sha1-MxVkNAO/J5BsqAqzfDbs6UQO8zA=
+
 gensync@^1.0.0-beta.1:
   version "1.0.0-beta.1"
   resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.1.tgz#58f4361ff987e5ff6e1e7a210827aa371eaac269"
   integrity sha512-r8EC6NO1sngH/zdD9fiRDLdcgnbayXah+mLgManTaIZJqEC1MZstmnox8KpnI2/fxQwrp5OpCOYWLp4rBl4Jcg==
 
+geojson-vt@^3.2.1:
+  version "3.2.1"
+  resolved "https://registry.yarnpkg.com/geojson-vt/-/geojson-vt-3.2.1.tgz#f8adb614d2c1d3f6ee7c4265cad4bbf3ad60c8b7"
+  integrity sha512-EvGQQi/zPrDA6zr6BnJD/YhwAkBP8nnJ9emh3EnHQKVMfg/MRVtPbMYdgVy/IaEmn4UfagD2a6fafPDL5hbtwg==
+
 get-caller-file@^1.0.1:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-1.0.3.tgz#f978fa4c90d1dfe7ff2d6beda2a515e713bdcf4a"
@@ -5659,6 +6728,11 @@
   resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e"
   integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==
 
+get-canvas-context@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/get-canvas-context/-/get-canvas-context-1.0.2.tgz#d6e7b50bc4e4c86357cd39f22647a84b73601e93"
+  integrity sha1-1ue1C8TkyGNXzTnyJkeoS3NgHpM=
+
 get-own-enumerable-property-symbols@^3.0.0:
   version "3.0.2"
   resolved "https://registry.yarnpkg.com/get-own-enumerable-property-symbols/-/get-own-enumerable-property-symbols-3.0.2.tgz#b5fde77f22cbe35f390b4e089922c50bce6ef664"
@@ -5710,6 +6784,393 @@
   dependencies:
     assert-plus "^1.0.0"
 
+gl-axes3d@^1.5.3:
+  version "1.5.3"
+  resolved "https://registry.yarnpkg.com/gl-axes3d/-/gl-axes3d-1.5.3.tgz#47e3dd6c21356a59349910ec01af58e28ea69fe9"
+  integrity sha512-KRYbguKQcDQ6PcB9g1pgqB8Ly4TY1DQODpPKiDTasyWJ8PxQk0t2Q7XoQQijNqvsguITCpVVCzNb5GVtIWiVlQ==
+  dependencies:
+    bit-twiddle "^1.0.2"
+    dup "^1.0.0"
+    extract-frustum-planes "^1.0.0"
+    gl-buffer "^2.1.2"
+    gl-mat4 "^1.2.0"
+    gl-shader "^4.2.1"
+    gl-state "^1.0.0"
+    gl-vao "^1.3.0"
+    gl-vec4 "^1.0.1"
+    glslify "^7.0.0"
+    robust-orientation "^1.1.3"
+    split-polygon "^1.0.0"
+    vectorize-text "^3.2.1"
+
+gl-buffer@^2.1.1, gl-buffer@^2.1.2:
+  version "2.1.2"
+  resolved "https://registry.yarnpkg.com/gl-buffer/-/gl-buffer-2.1.2.tgz#2db8d9c1a5527fba0cdb91289c206e882b889cdb"
+  integrity sha1-LbjZwaVSf7oM25EonCBuiCuInNs=
+  dependencies:
+    ndarray "^1.0.15"
+    ndarray-ops "^1.1.0"
+    typedarray-pool "^1.0.0"
+
+gl-cone3d@^1.5.2:
+  version "1.5.2"
+  resolved "https://registry.yarnpkg.com/gl-cone3d/-/gl-cone3d-1.5.2.tgz#66af5c33b7d5174034dfa3654a88e995998d92bc"
+  integrity sha512-1JNeHH4sUtUmDA4ZK7Om8/kShwb8IZVAsnxaaB7IPRJsNGciLj1sTpODrJGeMl41RNkex5kXD2SQFrzyEAR2Rw==
+  dependencies:
+    colormap "^2.3.1"
+    gl-buffer "^2.1.2"
+    gl-mat4 "^1.2.0"
+    gl-shader "^4.2.1"
+    gl-texture2d "^2.1.0"
+    gl-vao "^1.3.0"
+    gl-vec3 "^1.1.3"
+    glsl-inverse "^1.0.0"
+    glsl-out-of-range "^1.0.4"
+    glsl-specular-cook-torrance "^2.0.1"
+    glslify "^7.0.0"
+    ndarray "^1.0.18"
+
+gl-constants@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/gl-constants/-/gl-constants-1.0.0.tgz#597a504e364750ff50253aa35f8dea7af4a5d233"
+  integrity sha1-WXpQTjZHUP9QJTqjX43qevSl0jM=
+
+gl-contour2d@^1.1.7:
+  version "1.1.7"
+  resolved "https://registry.yarnpkg.com/gl-contour2d/-/gl-contour2d-1.1.7.tgz#ca330cf8449673a9ca0b3f6726c83f8d35c7a50c"
+  integrity sha512-GdebvJ9DtT3pJDpoE+eU2q+Wo9S3MijPpPz5arZbhK85w2bARmpFpVfPaDlZqWkB644W3BlH8TVyvAo1KE4Bhw==
+  dependencies:
+    binary-search-bounds "^2.0.4"
+    cdt2d "^1.0.0"
+    clean-pslg "^1.1.2"
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    glslify "^7.0.0"
+    iota-array "^1.0.0"
+    ndarray "^1.0.18"
+    surface-nets "^1.0.2"
+
+gl-error3d@^1.0.16:
+  version "1.0.16"
+  resolved "https://registry.yarnpkg.com/gl-error3d/-/gl-error3d-1.0.16.tgz#88a94952f5303d9cf5cb86806789a360777c5446"
+  integrity sha512-TGJewnKSp7ZnqGgG3XCF9ldrDbxZrO+OWlx6oIet4OdOM//n8xJ5isArnIV/sdPJnFbhfoLxWrW9f5fxHFRQ1A==
+  dependencies:
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    gl-vao "^1.3.0"
+    glsl-out-of-range "^1.0.4"
+    glslify "^7.0.0"
+
+gl-fbo@^2.0.5:
+  version "2.0.5"
+  resolved "https://registry.yarnpkg.com/gl-fbo/-/gl-fbo-2.0.5.tgz#0fa75a497cf787695530691c8f04abb6fb55fa22"
+  integrity sha1-D6daSXz3h2lVMGkcjwSrtvtV+iI=
+  dependencies:
+    gl-texture2d "^2.0.0"
+
+gl-format-compiler-error@^1.0.2:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/gl-format-compiler-error/-/gl-format-compiler-error-1.0.3.tgz#0c79b1751899ce9732e86240f090aa41e98471a8"
+  integrity sha1-DHmxdRiZzpcy6GJA8JCqQemEcag=
+  dependencies:
+    add-line-numbers "^1.0.1"
+    gl-constants "^1.0.0"
+    glsl-shader-name "^1.0.0"
+    sprintf-js "^1.0.3"
+
+gl-heatmap2d@^1.0.6:
+  version "1.0.6"
+  resolved "https://registry.yarnpkg.com/gl-heatmap2d/-/gl-heatmap2d-1.0.6.tgz#d0a2d9e8f5975534224b3b2be2ffe683f6b7e961"
+  integrity sha512-+agzSv4R5vsaH+AGYVz5RVzBK10amqAa+Bwj205F13JjNSGS91M1L9Yb8zssCv2FIjpP+1Mp73cFBYrQFfS1Jg==
+  dependencies:
+    binary-search-bounds "^2.0.4"
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    glslify "^7.0.0"
+    iota-array "^1.0.0"
+    typedarray-pool "^1.1.0"
+
+gl-line3d@1.2.1:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/gl-line3d/-/gl-line3d-1.2.1.tgz#632fc5b931a84a315995322b271aaf497e292609"
+  integrity sha512-eeb0+RI2ZBRqMYJK85SgsRiJK7c4aiOjcnirxv0830A3jmOc99snY3AbPcV8KvKmW0Yaf3KA4e+qNCbHiTOTnA==
+  dependencies:
+    binary-search-bounds "^2.0.4"
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    gl-texture2d "^2.1.0"
+    gl-vao "^1.3.0"
+    glsl-out-of-range "^1.0.4"
+    glslify "^7.0.0"
+    ndarray "^1.0.18"
+
+gl-mat2@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/gl-mat2/-/gl-mat2-1.0.1.tgz#142505730a5c2fe1e9f25d9ece3d0d6cc2710a30"
+  integrity sha1-FCUFcwpcL+Hp8l2ezj0NbMJxCjA=
+
+gl-mat3@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/gl-mat3/-/gl-mat3-1.0.0.tgz#89633219ca429379a16b9185d95d41713453b912"
+  integrity sha1-iWMyGcpCk3mha5GF2V1BcTRTuRI=
+
+gl-mat4@^1.0.0, gl-mat4@^1.0.1, gl-mat4@^1.0.2, gl-mat4@^1.0.3, gl-mat4@^1.1.2, gl-mat4@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/gl-mat4/-/gl-mat4-1.2.0.tgz#49d8a7636b70aa00819216635f4a3fd3f4669b26"
+  integrity sha512-sT5C0pwB1/e9G9AvAoLsoaJtbMGjfd/jfxo8jMCKqYYEnjZuFvqV5rehqar0538EmssjdDeiEWnKyBSTw7quoA==
+
+gl-matrix-invert@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/gl-matrix-invert/-/gl-matrix-invert-1.0.0.tgz#a36d7bde3654c4590a127ee7c68f6e13fea8c63d"
+  integrity sha1-o2173jZUxFkKEn7nxo9uE/6oxj0=
+  dependencies:
+    gl-mat2 "^1.0.0"
+    gl-mat3 "^1.0.0"
+    gl-mat4 "^1.0.0"
+
+gl-matrix@^3.2.1:
+  version "3.3.0"
+  resolved "https://registry.yarnpkg.com/gl-matrix/-/gl-matrix-3.3.0.tgz#232eef60b1c8b30a28cbbe75b2caf6c48fd6358b"
+  integrity sha512-COb7LDz+SXaHtl/h4LeaFcNdJdAQSDeVqjiIihSXNrkWObZLhDI4hIkZC11Aeqp7bcE72clzB0BnDXr2SmslRA==
+
+gl-mesh3d@^2.3.1:
+  version "2.3.1"
+  resolved "https://registry.yarnpkg.com/gl-mesh3d/-/gl-mesh3d-2.3.1.tgz#087a93c5431df923570ca51cfc691bab0d21a6b8"
+  integrity sha512-pXECamyGgu4/9HeAQSE5OEUuLBGS1aq9V4BCsTcxsND4fNLaajEkYKUz/WY2QSYElqKdsMBVsldGiKRKwlybqA==
+  dependencies:
+    barycentric "^1.0.1"
+    colormap "^2.3.1"
+    gl-buffer "^2.1.2"
+    gl-mat4 "^1.2.0"
+    gl-shader "^4.2.1"
+    gl-texture2d "^2.1.0"
+    gl-vao "^1.3.0"
+    glsl-out-of-range "^1.0.4"
+    glsl-specular-cook-torrance "^2.0.1"
+    glslify "^7.0.0"
+    ndarray "^1.0.18"
+    normals "^1.1.0"
+    polytope-closest-point "^1.0.0"
+    simplicial-complex-contour "^1.0.2"
+    typedarray-pool "^1.1.0"
+
+gl-plot2d@^1.4.4:
+  version "1.4.4"
+  resolved "https://registry.yarnpkg.com/gl-plot2d/-/gl-plot2d-1.4.4.tgz#2dbe40776051436baf0947f828ee2c8c303d2bdb"
+  integrity sha512-0UhKiiqeampLtydv6NMNrKEilc0Ui5oaJtvHLbLZ5u/1ttT1XjOY5Yk8LzfqozA/No4a9omxjSKnH+tvSn+rQQ==
+  dependencies:
+    binary-search-bounds "^2.0.4"
+    gl-buffer "^2.1.2"
+    gl-select-static "^2.0.6"
+    gl-shader "^4.2.1"
+    glsl-inverse "^1.0.0"
+    glslify "^7.0.0"
+    text-cache "^4.2.2"
+
+gl-plot3d@^2.4.5:
+  version "2.4.5"
+  resolved "https://registry.yarnpkg.com/gl-plot3d/-/gl-plot3d-2.4.5.tgz#962e99c3e55862e0ad77e99114566eadb28bf224"
+  integrity sha512-cKAqMXFRHTCFxH8r1/ACdk5hyfnA9djfiAM8zVQrqu0qLEttUu0i1fq0pr+d5m0HPuNcK8wEc4F3VjL2hrDcGQ==
+  dependencies:
+    "3d-view" "^2.0.0"
+    a-big-triangle "^1.0.3"
+    gl-axes3d "^1.5.3"
+    gl-fbo "^2.0.5"
+    gl-mat4 "^1.2.0"
+    gl-select-static "^2.0.6"
+    gl-shader "^4.2.1"
+    gl-spikes3d "^1.0.10"
+    glslify "^7.0.0"
+    has-passive-events "^1.0.0"
+    is-mobile "^2.2.1"
+    mouse-change "^1.4.0"
+    mouse-event-offset "^3.0.2"
+    mouse-wheel "^1.2.0"
+    ndarray "^1.0.18"
+    right-now "^1.0.0"
+
+gl-pointcloud2d@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/gl-pointcloud2d/-/gl-pointcloud2d-1.0.3.tgz#f37e215f21ccb2e17f0604664e99fc3d6a4e611d"
+  integrity sha512-OS2e1irvJXVRpg/GziXj10xrFJm9kkRfFoB6BLUvkjCQV7ZRNNcs2CD+YSK1r0gvMwTg2T3lfLM3UPwNtz+4Xw==
+  dependencies:
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    glslify "^7.0.0"
+    typedarray-pool "^1.1.0"
+
+gl-quat@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/gl-quat/-/gl-quat-1.0.0.tgz#0945ec923386f45329be5dc357b1c8c2d47586c5"
+  integrity sha1-CUXskjOG9FMpvl3DV7HIwtR1hsU=
+  dependencies:
+    gl-mat3 "^1.0.0"
+    gl-vec3 "^1.0.3"
+    gl-vec4 "^1.0.0"
+
+gl-scatter3d@^1.2.3:
+  version "1.2.3"
+  resolved "https://registry.yarnpkg.com/gl-scatter3d/-/gl-scatter3d-1.2.3.tgz#83d63700ec2fe4e95b3d1cd613e86de9a6b5f603"
+  integrity sha512-nXqPlT1w5Qt51dTksj+DUqrZqwWAEWg0PocsKcoDnVNv0X8sGA+LBZ0Y+zrA+KNXUL0PPCX9WR9cF2uJAZl1Sw==
+  dependencies:
+    gl-buffer "^2.1.2"
+    gl-mat4 "^1.2.0"
+    gl-shader "^4.2.1"
+    gl-vao "^1.3.0"
+    glsl-out-of-range "^1.0.4"
+    glslify "^7.0.0"
+    is-string-blank "^1.0.1"
+    typedarray-pool "^1.1.0"
+    vectorize-text "^3.2.1"
+
+gl-select-box@^1.0.4:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/gl-select-box/-/gl-select-box-1.0.4.tgz#47c11caa2b84f81e8bbfde08c6e39eeebb53d3d8"
+  integrity sha512-mKsCnglraSKyBbQiGq0Ila0WF+m6Tr+EWT2yfaMn/Sh9aMHq5Wt0F/l6Cf/Ed3CdERq5jHWAY5yxLviZteYu2w==
+  dependencies:
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    glslify "^7.0.0"
+
+gl-select-static@^2.0.6:
+  version "2.0.6"
+  resolved "https://registry.yarnpkg.com/gl-select-static/-/gl-select-static-2.0.6.tgz#43194a348613a693d4097e5ab9ceb0685003f036"
+  integrity sha512-p4DmBG1DMo/47/fV3oqPcU6uTqHy0eI1vATH1fm8OVDqlzWnLv3786tdEunZWG6Br7DUdH6NgWhuy4gAlt+TAQ==
+  dependencies:
+    bit-twiddle "^1.0.2"
+    cwise "^1.0.10"
+    gl-fbo "^2.0.5"
+    ndarray "^1.0.18"
+    typedarray-pool "^1.1.0"
+
+gl-shader@^4.2.1:
+  version "4.2.1"
+  resolved "https://registry.yarnpkg.com/gl-shader/-/gl-shader-4.2.1.tgz#bc9b808e9293c51b668e88de615b0c113708dc2f"
+  integrity sha1-vJuAjpKTxRtmjojeYVsMETcI3C8=
+  dependencies:
+    gl-format-compiler-error "^1.0.2"
+    weakmap-shim "^1.1.0"
+
+gl-spikes2d@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/gl-spikes2d/-/gl-spikes2d-1.0.2.tgz#ef8dbcff6c7451dec2b751d7a3c593d09ad5457f"
+  integrity sha512-QVeOZsi9nQuJJl7NB3132CCv5KA10BWxAY2QgJNsKqbLsG53B/TrGJpjIAohnJftdZ4fT6b3ZojWgeaXk8bOOA==
+
+gl-spikes3d@^1.0.10:
+  version "1.0.10"
+  resolved "https://registry.yarnpkg.com/gl-spikes3d/-/gl-spikes3d-1.0.10.tgz#e3b2b677a6f51750f23c064447af4f093da79305"
+  integrity sha512-lT3xroowOFxMvlhT5Mof76B2TE02l5zt/NIWljhczV2FFHgIVhA4jMrd5dIv1so1RXMBDJIKu0uJI3QKliDVLg==
+  dependencies:
+    gl-buffer "^2.1.2"
+    gl-shader "^4.2.1"
+    gl-vao "^1.3.0"
+    glslify "^7.0.0"
+
+gl-state@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/gl-state/-/gl-state-1.0.0.tgz#262faa75835b0b9c532c12f38adc425d1d30cd17"
+  integrity sha1-Ji+qdYNbC5xTLBLzitxCXR0wzRc=
+  dependencies:
+    uniq "^1.0.0"
+
+gl-streamtube3d@^1.4.1:
+  version "1.4.1"
+  resolved "https://registry.yarnpkg.com/gl-streamtube3d/-/gl-streamtube3d-1.4.1.tgz#bd2b725e00aa96989ce34b06ebf66a76f93e35ae"
+  integrity sha512-rH02v00kgwgdpkXVo7KsSoPp38bIAYR9TE1iONjcQ4cQAlDhrGRauqT/P5sUaOIzs17A2DxWGcXM+EpNQs9pUA==
+  dependencies:
+    gl-cone3d "^1.5.2"
+    gl-vec3 "^1.1.3"
+    gl-vec4 "^1.0.1"
+    glsl-inverse "^1.0.0"
+    glsl-out-of-range "^1.0.4"
+    glsl-specular-cook-torrance "^2.0.1"
+    glslify "^7.0.0"
+
+gl-surface3d@^1.5.2:
+  version "1.5.2"
+  resolved "https://registry.yarnpkg.com/gl-surface3d/-/gl-surface3d-1.5.2.tgz#a283b98d3473fca4e8552f0f6ac40a6f0dc88b00"
+  integrity sha512-rWSQwEQDkB0T5CDEDFJwJc4VgwwJaAyFRSJ92NJlrTSwDlsEsWdzG9+APx6FWJMwkOpIoZGWqv+csswK2kMMLQ==
+  dependencies:
+    binary-search-bounds "^2.0.4"
+    bit-twiddle "^1.0.2"
+    colormap "^2.3.1"
+    dup "^1.0.0"
+    gl-buffer "^2.1.2"
+    gl-mat4 "^1.2.0"
+    gl-shader "^4.2.1"
+    gl-texture2d "^2.1.0"
+    gl-vao "^1.3.0"
+    glsl-out-of-range "^1.0.4"
+    glsl-specular-beckmann "^1.1.2"
+    glslify "^7.0.0"
+    ndarray "^1.0.18"
+    ndarray-gradient "^1.0.0"
+    ndarray-ops "^1.2.2"
+    ndarray-pack "^1.2.1"
+    ndarray-scratch "^1.2.0"
+    surface-nets "^1.0.2"
+    typedarray-pool "^1.1.0"
+
+gl-text@^1.1.8:
+  version "1.1.8"
+  resolved "https://registry.yarnpkg.com/gl-text/-/gl-text-1.1.8.tgz#67a19bec72915acc422300aad8f727a09f98b550"
+  integrity sha512-whnq9DEFYbW92C4ONwk2eT0YkzmVPHoADnEtuzMOmit87XhgAhBrNs3lK9EgGjU/MoWYvlF6RkI8Kl7Yuo1hUw==
+  dependencies:
+    bit-twiddle "^1.0.2"
+    color-normalize "^1.5.0"
+    css-font "^1.2.0"
+    detect-kerning "^2.1.2"
+    es6-weak-map "^2.0.3"
+    flatten-vertex-data "^1.0.2"
+    font-atlas "^2.1.0"
+    font-measure "^1.2.2"
+    gl-util "^3.1.2"
+    is-plain-obj "^1.1.0"
+    object-assign "^4.1.1"
+    parse-rect "^1.2.0"
+    parse-unit "^1.0.1"
+    pick-by-alias "^1.2.0"
+    regl "^1.3.11"
+    to-px "^1.0.1"
+    typedarray-pool "^1.1.0"
+
+gl-texture2d@^2.0.0, gl-texture2d@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/gl-texture2d/-/gl-texture2d-2.1.0.tgz#ff6824e7e7c31a8ba6fdcdbe9e5c695d7e2187c7"
+  integrity sha1-/2gk5+fDGoum/c2+nlxpXX4hh8c=
+  dependencies:
+    ndarray "^1.0.15"
+    ndarray-ops "^1.2.2"
+    typedarray-pool "^1.1.0"
+
+gl-util@^3.1.2:
+  version "3.1.3"
+  resolved "https://registry.yarnpkg.com/gl-util/-/gl-util-3.1.3.tgz#1e9a724f844b802597c6e30565d4c1e928546861"
+  integrity sha512-dvRTggw5MSkJnCbh74jZzSoTOGnVYK+Bt+Ckqm39CVcl6+zSsxqWk4lr5NKhkqXHL6qvZAU9h17ZF8mIskY9mA==
+  dependencies:
+    is-browser "^2.0.1"
+    is-firefox "^1.0.3"
+    is-plain-obj "^1.1.0"
+    number-is-integer "^1.0.1"
+    object-assign "^4.1.0"
+    pick-by-alias "^1.2.0"
+    weak-map "^1.0.5"
+
+gl-vao@^1.2.0, gl-vao@^1.3.0:
+  version "1.3.0"
+  resolved "https://registry.yarnpkg.com/gl-vao/-/gl-vao-1.3.0.tgz#e9e92aa95588cab9d5c2f04b693440c3df691923"
+  integrity sha1-6ekqqVWIyrnVwvBLaTRAw99pGSM=
+
+gl-vec3@^1.0.2, gl-vec3@^1.0.3, gl-vec3@^1.1.3:
+  version "1.1.3"
+  resolved "https://registry.yarnpkg.com/gl-vec3/-/gl-vec3-1.1.3.tgz#a47c62f918774a06cbed1b65bcd0288ecbb03826"
+  integrity sha512-jduKUqT0SGH02l8Yl+mV1yVsDfYgQAJyXGxkJQGyxPLHRiW25DwVIRPt6uvhrEMHftJfqhqKthRcyZqNEl9Xdw==
+
+gl-vec4@^1.0.0, gl-vec4@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/gl-vec4/-/gl-vec4-1.0.1.tgz#97d96878281b14b532cbce101785dfd1cb340964"
+  integrity sha1-l9loeCgbFLUyy84QF4Xf0cs0CWQ=
+
 glob-parent@^3.1.0:
   version "3.1.0"
   resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-3.1.0.tgz#9e6af6299d8d3bd2bd40430832bd113df906c5ae"
@@ -5730,7 +7191,7 @@
   resolved "https://registry.yarnpkg.com/glob-to-regexp/-/glob-to-regexp-0.3.0.tgz#8c5a1494d2066c570cc3bfe4496175acc4d502ab"
   integrity sha1-jFoUlNIGbFcMw7/kSWF1rMTVAqs=
 
-glob@^7.0.3, glob@^7.1.1, glob@^7.1.2, glob@^7.1.3, glob@^7.1.4, glob@^7.1.6:
+glob@^7.0.3, glob@^7.1.1, glob@^7.1.2, glob@^7.1.3, glob@^7.1.4, glob@^7.1.6, glob@~7.1.6:
   version "7.1.6"
   resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.6.tgz#141f33b81a7c2492e125594307480c46679278a6"
   integrity sha512-LwaxwyZ72Lk7vZINtNNrywX0ZuLyStrdDtabefZKAY5ZGJhVtgdznluResxNmPitE0SAO+O26sWTHeKSI2wMBA==
@@ -5822,6 +7283,163 @@
     pify "^4.0.1"
     slash "^2.0.0"
 
+glsl-inject-defines@^1.0.1:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/glsl-inject-defines/-/glsl-inject-defines-1.0.3.tgz#dd1aacc2c17fcb2bd3fc32411c6633d0d7b60fd4"
+  integrity sha1-3RqswsF/yyvT/DJBHGYz0Ne2D9Q=
+  dependencies:
+    glsl-token-inject-block "^1.0.0"
+    glsl-token-string "^1.0.1"
+    glsl-tokenizer "^2.0.2"
+
+glsl-inverse@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/glsl-inverse/-/glsl-inverse-1.0.0.tgz#12c0b1d065f558444d1e6feaf79b5ddf8a918ae6"
+  integrity sha1-EsCx0GX1WERNHm/q95td34qRiuY=
+
+glsl-out-of-range@^1.0.4:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/glsl-out-of-range/-/glsl-out-of-range-1.0.4.tgz#3d73d083bc9ecc73efd45dfc7063c29e92c9c873"
+  integrity sha512-fCcDu2LCQ39VBvfe1FbhuazXEf0CqMZI9OYXrYlL6uUARG48CTAbL04+tZBtVM0zo1Ljx4OLu2AxNquq++lxWQ==
+
+glsl-resolve@0.0.1:
+  version "0.0.1"
+  resolved "https://registry.yarnpkg.com/glsl-resolve/-/glsl-resolve-0.0.1.tgz#894bef73910d792c81b5143180035d0a78af76d3"
+  integrity sha1-iUvvc5ENeSyBtRQxgANdCnivdtM=
+  dependencies:
+    resolve "^0.6.1"
+    xtend "^2.1.2"
+
+glsl-shader-name@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/glsl-shader-name/-/glsl-shader-name-1.0.0.tgz#a2c30b3ba73499befb0cc7184d7c7733dd4b487d"
+  integrity sha1-osMLO6c0mb77DMcYTXx3M91LSH0=
+  dependencies:
+    atob-lite "^1.0.0"
+    glsl-tokenizer "^2.0.2"
+
+glsl-specular-beckmann@^1.1.1, glsl-specular-beckmann@^1.1.2:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/glsl-specular-beckmann/-/glsl-specular-beckmann-1.1.2.tgz#fce9056933ecdf2456278376a54d082893e775f1"
+  integrity sha1-/OkFaTPs3yRWJ4N2pU0IKJPndfE=
+
+glsl-specular-cook-torrance@^2.0.1:
+  version "2.0.1"
+  resolved "https://registry.yarnpkg.com/glsl-specular-cook-torrance/-/glsl-specular-cook-torrance-2.0.1.tgz#a891cc06c8c7b4f4728702b4824fdacbb967d78f"
+  integrity sha1-qJHMBsjHtPRyhwK0gk/ay7ln148=
+  dependencies:
+    glsl-specular-beckmann "^1.1.1"
+
+glsl-token-assignments@^2.0.0:
+  version "2.0.2"
+  resolved "https://registry.yarnpkg.com/glsl-token-assignments/-/glsl-token-assignments-2.0.2.tgz#a5d82ab78499c2e8a6b83cb69495e6e665ce019f"
+  integrity sha1-pdgqt4SZwuimuDy2lJXm5mXOAZ8=
+
+glsl-token-defines@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/glsl-token-defines/-/glsl-token-defines-1.0.0.tgz#cb892aa959936231728470d4f74032489697fa9d"
+  integrity sha1-y4kqqVmTYjFyhHDU90AySJaX+p0=
+  dependencies:
+    glsl-tokenizer "^2.0.0"
+
+glsl-token-depth@^1.1.0, glsl-token-depth@^1.1.1:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/glsl-token-depth/-/glsl-token-depth-1.1.2.tgz#23c5e30ee2bd255884b4a28bc850b8f791e95d84"
+  integrity sha1-I8XjDuK9JViEtKKLyFC495HpXYQ=
+
+glsl-token-descope@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/glsl-token-descope/-/glsl-token-descope-1.0.2.tgz#0fc90ab326186b82f597b2e77dc9e21efcd32076"
+  integrity sha1-D8kKsyYYa4L1l7LnfcniHvzTIHY=
+  dependencies:
+    glsl-token-assignments "^2.0.0"
+    glsl-token-depth "^1.1.0"
+    glsl-token-properties "^1.0.0"
+    glsl-token-scope "^1.1.0"
+
+glsl-token-inject-block@^1.0.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/glsl-token-inject-block/-/glsl-token-inject-block-1.1.0.tgz#e1015f5980c1091824adaa2625f1dfde8bd00034"
+  integrity sha1-4QFfWYDBCRgkraomJfHf3ovQADQ=
+
+glsl-token-properties@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/glsl-token-properties/-/glsl-token-properties-1.0.1.tgz#483dc3d839f0d4b5c6171d1591f249be53c28a9e"
+  integrity sha1-SD3D2Dnw1LXGFx0VkfJJvlPCip4=
+
+glsl-token-scope@^1.1.0, glsl-token-scope@^1.1.1:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/glsl-token-scope/-/glsl-token-scope-1.1.2.tgz#a1728e78df24444f9cb93fd18ef0f75503a643b1"
+  integrity sha1-oXKOeN8kRE+cuT/RjvD3VQOmQ7E=
+
+glsl-token-string@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/glsl-token-string/-/glsl-token-string-1.0.1.tgz#59441d2f857de7c3449c945666021ece358e48ec"
+  integrity sha1-WUQdL4V958NEnJRWZgIezjWOSOw=
+
+glsl-token-whitespace-trim@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/glsl-token-whitespace-trim/-/glsl-token-whitespace-trim-1.0.0.tgz#46d1dfe98c75bd7d504c05d7d11b1b3e9cc93b10"
+  integrity sha1-RtHf6Yx1vX1QTAXX0RsbPpzJOxA=
+
+glsl-tokenizer@^2.0.0, glsl-tokenizer@^2.0.2:
+  version "2.1.5"
+  resolved "https://registry.yarnpkg.com/glsl-tokenizer/-/glsl-tokenizer-2.1.5.tgz#1c2e78c16589933c274ba278d0a63b370c5fee1a"
+  integrity sha512-XSZEJ/i4dmz3Pmbnpsy3cKh7cotvFlBiZnDOwnj/05EwNp2XrhQ4XKJxT7/pDt4kp4YcpRSKz8eTV7S+mwV6MA==
+  dependencies:
+    through2 "^0.6.3"
+
+glslify-bundle@^5.0.0:
+  version "5.1.1"
+  resolved "https://registry.yarnpkg.com/glslify-bundle/-/glslify-bundle-5.1.1.tgz#30d2ddf2e6b935bf44d1299321e3b729782c409a"
+  integrity sha512-plaAOQPv62M1r3OsWf2UbjN0hUYAB7Aph5bfH58VxJZJhloRNbxOL9tl/7H71K7OLJoSJ2ZqWOKk3ttQ6wy24A==
+  dependencies:
+    glsl-inject-defines "^1.0.1"
+    glsl-token-defines "^1.0.0"
+    glsl-token-depth "^1.1.1"
+    glsl-token-descope "^1.0.2"
+    glsl-token-scope "^1.1.1"
+    glsl-token-string "^1.0.1"
+    glsl-token-whitespace-trim "^1.0.0"
+    glsl-tokenizer "^2.0.2"
+    murmurhash-js "^1.0.0"
+    shallow-copy "0.0.1"
+
+glslify-deps@^1.2.5:
+  version "1.3.1"
+  resolved "https://registry.yarnpkg.com/glslify-deps/-/glslify-deps-1.3.1.tgz#dfa6962322454a91ecc4de25b5e710415b0c89ad"
+  integrity sha512-Ogm179MCazwIRyEqs3g3EOY4Y3XIAa0yl8J5RE9rJC6QH1w8weVOp2RZu0mvnYy/2xIas1w166YR2eZdDkWQxg==
+  dependencies:
+    "@choojs/findup" "^0.2.0"
+    events "^1.0.2"
+    glsl-resolve "0.0.1"
+    glsl-tokenizer "^2.0.0"
+    graceful-fs "^4.1.2"
+    inherits "^2.0.1"
+    map-limit "0.0.1"
+    resolve "^1.0.0"
+
+glslify@^7.0.0:
+  version "7.0.0"
+  resolved "https://registry.yarnpkg.com/glslify/-/glslify-7.0.0.tgz#10d5db9541ee07c6548ea55c679edda20307653d"
+  integrity sha512-yw8jDQIe9FlSH5NiZEqSAsCPj9HI7nhXgXLAgSv2Nm9eBPsFJmyN9+rNwbiozJapcj9xtc/71rMYlN9cxp1B8Q==
+  dependencies:
+    bl "^1.0.0"
+    concat-stream "^1.5.2"
+    duplexify "^3.4.5"
+    falafel "^2.1.0"
+    from2 "^2.3.0"
+    glsl-resolve "0.0.1"
+    glsl-token-whitespace-trim "^1.0.0"
+    glslify-bundle "^5.0.0"
+    glslify-deps "^1.2.5"
+    minimist "^1.2.0"
+    resolve "^1.1.5"
+    stack-trace "0.0.9"
+    static-eval "^2.0.0"
+    through2 "^2.0.1"
+    xtend "^4.0.0"
+
 got@^9.6.0:
   version "9.6.0"
   resolved "https://registry.yarnpkg.com/got/-/got-9.6.0.tgz#edf45e7d67f99545705de1f7bbeeeb121765ed85"
@@ -5849,6 +7467,11 @@
   resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.4.tgz#2256bde14d3632958c465ebc96dc467ca07a29fb"
   integrity sha512-WjKPNJF79dtJAVniUlGGWHYGz2jWxT6VhN/4m1NdkbZ2nOsEF+cI1Edgql5zCRhs/VsQYRvrXctxktVXZUkixw==
 
+grid-index@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/grid-index/-/grid-index-1.1.0.tgz#97f8221edec1026c8377b86446a7c71e79522ea7"
+  integrity sha512-HZRwumpOGUrHyxO5bqKZL0B0GlUpwtCAzZ42sgxUPniu33R1LSFH5yrIcBCHjkctCAh3mtWKcKd9J4vDDdeVHA==
+
 growly@^1.3.0:
   version "1.3.0"
   resolved "https://registry.yarnpkg.com/growly/-/growly-1.3.0.tgz#f10748cbe76af964b7c96c93c6bcc28af120c081"
@@ -5923,6 +7546,20 @@
   resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b"
   integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==
 
+has-hover@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/has-hover/-/has-hover-1.0.1.tgz#3d97437aeb199c62b8ac08acbdc53d3bc52c17f7"
+  integrity sha1-PZdDeusZnGK4rAisvcU9O8UsF/c=
+  dependencies:
+    is-browser "^2.0.1"
+
+has-passive-events@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/has-passive-events/-/has-passive-events-1.0.0.tgz#75fc3dc6dada182c58f24ebbdc018276d1ea3515"
+  integrity sha512-2vSj6IeIsgvsRMyeQ0JaCX5Q3lX4zMn5HpoVc7MEhQ6pv8Iq9rsXjsp+E5ZwaT7T0xhMT0KmU8gtt1EFVdbJiw==
+  dependencies:
+    is-browser "^2.0.1"
+
 has-symbols@^1.0.0, has-symbols@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.1.tgz#9f5214758a44196c406d9bd76cebf81ec2dd31e8"
@@ -5964,7 +7601,7 @@
   resolved "https://registry.yarnpkg.com/has-yarn/-/has-yarn-2.1.0.tgz#137e11354a7b5bf11aa5cb649cf0c6f3ff2b2e77"
   integrity sha512-UqBRqi4ju7T+TqGNdqAO0PaSVGsDGJUBQvk9eUWNGRY1CFGDzYhLWoM7JQEemnlvVcv/YEmc2wNW8BC24EnUsw==
 
-has@^1.0.0, has@^1.0.3:
+has@^1.0.0, has@^1.0.3, has@~1.0.3:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/has/-/has-1.0.3.tgz#722d7cbfc1f6aa8241f16dd814e011e1f41e8796"
   integrity sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==
@@ -6055,6 +7692,11 @@
   resolved "https://registry.yarnpkg.com/hsla-regex/-/hsla-regex-1.0.0.tgz#c1ce7a3168c8c6614033a4b5f7877f3b225f9c38"
   integrity sha1-wc56MWjIxmFAM6S194d/OyJfnDg=
 
+hsluv@^0.0.3:
+  version "0.0.3"
+  resolved "https://registry.yarnpkg.com/hsluv/-/hsluv-0.0.3.tgz#829107dafb4a9f8b52a1809ed02e091eade6754c"
+  integrity sha1-gpEH2vtKn4tSoYCe0C4JHq3mdUw=
+
 html-comment-regex@^1.1.0:
   version "1.1.2"
   resolved "https://registry.yarnpkg.com/html-comment-regex/-/html-comment-regex-1.1.2.tgz#97d4688aeb5c81886a364faa0cad1dda14d433a7"
@@ -6210,7 +7852,7 @@
   dependencies:
     harmony-reflect "^1.4.6"
 
-ieee754@^1.1.4:
+ieee754@^1.1.12, ieee754@^1.1.4:
   version "1.1.13"
   resolved "https://registry.yarnpkg.com/ieee754/-/ieee754-1.1.13.tgz#ec168558e95aa181fd87d37f55c32bbcb6708b84"
   integrity sha512-4vf7I2LYV/HaWerSo3XmlMkp5eZ83i+/CDluXi/IGTs/O1sejBNhTtnxzmRZfvOUqj7lZjqHkeTvpgSFDlWZTg==
@@ -6235,6 +7877,15 @@
   resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.1.4.tgz#84b7b3dbe64552b6ef0eca99f6743dbec6d97adf"
   integrity sha512-MzbUSahkTW1u7JpKKjY7LCARd1fU5W2rLdxlM4kdkayuCwZImjkpluF9CM1aLewYJguPDqewLam18Y6AU69A8A==
 
+image-palette@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/image-palette/-/image-palette-2.1.0.tgz#d976525a1df75964ca125d2dba2741e92905547f"
+  integrity sha512-3ImSEWD26+xuQFdP0RWR4WSXadZwvgrFhjGNpMEapTG1tf2XrBFS2dlKK5hNgH4UIaSQlSUFRn1NeA+zULIWbQ==
+  dependencies:
+    color-id "^1.1.0"
+    pxls "^2.0.0"
+    quantize "^1.0.2"
+
 image-size@~0.5.0:
   version "0.5.5"
   resolved "https://registry.yarnpkg.com/image-size/-/image-size-0.5.5.tgz#09dfd4ab9d20e29eb1c3e80b8990378df9e3cb9c"
@@ -6308,6 +7959,14 @@
   resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea"
   integrity sha1-khi5srkoojixPcT7a21XbyMUU+o=
 
+incremental-convex-hull@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/incremental-convex-hull/-/incremental-convex-hull-1.0.1.tgz#51428c14cb9d9a6144bfe69b2851fb377334be1e"
+  integrity sha1-UUKMFMudmmFEv+abKFH7N3M0vh4=
+  dependencies:
+    robust-orientation "^1.1.2"
+    simplicial-complex "^1.0.0"
+
 indent-string@^3.0.0:
   version "3.2.0"
   resolved "https://registry.yarnpkg.com/indent-string/-/indent-string-3.2.0.tgz#4a5fd6d27cc332f37e5419a504dbb837105c9289"
@@ -6336,7 +7995,7 @@
     once "^1.3.0"
     wrappy "1"
 
-inherits@2, inherits@2.0.4, inherits@^2.0.1, inherits@^2.0.3, inherits@~2.0.1, inherits@~2.0.3:
+inherits@2, inherits@2.0.4, inherits@^2.0.1, inherits@^2.0.3, inherits@~2.0.1, inherits@~2.0.3, inherits@~2.0.4:
   version "2.0.4"
   resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c"
   integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==
@@ -6416,6 +8075,13 @@
   resolved "https://registry.yarnpkg.com/interpret/-/interpret-1.2.0.tgz#d5061a6224be58e8083985f5014d844359576296"
   integrity sha512-mT34yGKMNceBQUoVn7iCDKDntA7SC6gycMAWzGx1z/CMCTV7b2AAtXlo3nRyHZ1FelRkQbQjprHSYGwzLtkVbw==
 
+interval-tree-1d@^1.0.1:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/interval-tree-1d/-/interval-tree-1d-1.0.3.tgz#8fdbde02b6b2c7dbdead636bcbed8e08710d85c1"
+  integrity sha1-j9veArayx9verWNry+2OCHENhcE=
+  dependencies:
+    binary-search-bounds "^1.0.0"
+
 invariant@^2.2.2, invariant@^2.2.4:
   version "2.2.4"
   resolved "https://registry.yarnpkg.com/invariant/-/invariant-2.2.4.tgz#610f3c92c9359ce1db616e538008d23ff35158e6"
@@ -6428,6 +8094,16 @@
   resolved "https://registry.yarnpkg.com/invert-kv/-/invert-kv-2.0.0.tgz#7393f5afa59ec9ff5f67a27620d11c226e3eec02"
   integrity sha512-wPVv/y/QQ/Uiirj/vh3oP+1Ww+AWehmi1g5fFWGPF6IpCBCDVrhgHRMvrLfdYcwDh3QJbGXDW4JAuzxElLSqKA==
 
+invert-permutation@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/invert-permutation/-/invert-permutation-1.0.0.tgz#a0a78042eadb36bc17551e787efd1439add54933"
+  integrity sha1-oKeAQurbNrwXVR54fv0UOa3VSTM=
+
+iota-array@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/iota-array/-/iota-array-1.0.0.tgz#81ef57fe5d05814cd58c2483632a99c30a0e8087"
+  integrity sha1-ge9X/l0FgUzVjCSDYyqZwwoOgIc=
+
 ip-regex@^2.1.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/ip-regex/-/ip-regex-2.1.0.tgz#fa78bf5d2e6913c911ce9f819ee5146bb6d844e9"
@@ -6500,6 +8176,11 @@
   resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.3.2.tgz#4574a2ae56f7ab206896fb431eaeed066fdf8f03"
   integrity sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ==
 
+is-base64@^0.1.0:
+  version "0.1.0"
+  resolved "https://registry.yarnpkg.com/is-base64/-/is-base64-0.1.0.tgz#a6f20610c6ef4863a51cba32bc0222544b932622"
+  integrity sha512-WRRyllsGXJM7ZN7gPTCCQ/6wNPTRDwiWdPK66l5sJzcU/oOzcIcRRf0Rux8bkpox/1yjt0F6VJRsQOIG2qz5sg==
+
 is-binary-path@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/is-binary-path/-/is-binary-path-1.0.1.tgz#75f16642b480f187a711c814161fd3a4a7655898"
@@ -6514,16 +8195,31 @@
   dependencies:
     binary-extensions "^2.0.0"
 
+is-blob@^2.0.1:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/is-blob/-/is-blob-2.1.0.tgz#e36cd82c90653f1e1b930f11baf9c64216a05385"
+  integrity sha512-SZ/fTft5eUhQM6oF/ZaASFDEdbFVe89Imltn9uZr03wdKMcWNVYSMjQPFtg05QuNkt5l5c135ElvXEQG0rk4tw==
+
+is-browser@^2.0.1, is-browser@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/is-browser/-/is-browser-2.1.0.tgz#fc084d59a5fced307d6708c59356bad7007371a9"
+  integrity sha512-F5rTJxDQ2sW81fcfOR1GnCXT6sVJC104fCyfj+mjpwNEwaPYSn5fte5jiHmBg3DHsIoL/l8Kvw5VN5SsTRcRFQ==
+
 is-buffer@^1.0.2, is-buffer@^1.1.5:
   version "1.1.6"
   resolved "https://registry.yarnpkg.com/is-buffer/-/is-buffer-1.1.6.tgz#efaa2ea9daa0d7ab2ea13a97b2b8ad51fefbe8be"
   integrity sha512-NcdALwpXkTm5Zvvbk7owOUSvVvBKDgKP5/ewfXEznmQFfs4ZRmanOeKBTjRVjka3QFoN6XJ+9F3USqfHqTaU5w==
 
-is-buffer@^2.0.2:
+is-buffer@^2.0.2, is-buffer@^2.0.3:
   version "2.0.4"
   resolved "https://registry.yarnpkg.com/is-buffer/-/is-buffer-2.0.4.tgz#3e572f23c8411a5cfd9557c849e3665e0b290623"
   integrity sha512-Kq1rokWXOPXWuaMAqZiJW4XxsmD9zGx9q4aePabbn3qCRGedtH7Cm+zV8WETitMfu1wdh+Rvd6w5egwSngUX2A==
 
+is-callable@^1.1.3:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.2.0.tgz#83336560b54a38e35e3a2df7afd0454d691468bb"
+  integrity sha512-pyVD9AaGLxtg6srb2Ng6ynWJqkHU9bEM087AKck0w8QwDarTfNcpIYoU8x8Hv2Icm8u6kFJM18Dag8lyqGkviw==
+
 is-callable@^1.1.4, is-callable@^1.1.5:
   version "1.1.5"
   resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.1.5.tgz#f7e46b596890456db74e7f6e976cb3273d06faab"
@@ -6617,6 +8313,21 @@
   resolved "https://registry.yarnpkg.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2"
   integrity sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=
 
+is-finite@^1.0.1:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/is-finite/-/is-finite-1.1.0.tgz#904135c77fb42c0641d6aa1bcdbc4daa8da082f3"
+  integrity sha512-cdyMtqX/BOqqNBBiKlIVkytNHm49MtMlYyn1zxzvJKWmFMlGzm+ry5BBfYyeY9YmNKbRSo/o7OX9w9ale0wg3w==
+
+is-firefox@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/is-firefox/-/is-firefox-1.0.3.tgz#2a2a1567783a417f6e158323108f3861b0918562"
+  integrity sha1-KioVZ3g6QX9uFYMjEI84YbCRhWI=
+
+is-float-array@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/is-float-array/-/is-float-array-1.0.0.tgz#96d67b1cbadf47ab1e05be208933acd386978a09"
+  integrity sha512-4ew1Sx6B6kEAl3T3NOM0yB94J3NZnBdNt4paw0e8nY73yHHTeTEhyQ3Lj7EQEnv5LD+GxNTaT4L46jcKjjpLiQ==
+
 is-fullwidth-code-point@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz#ef9e31386f031a7f0d643af82fde50c457ef00cb"
@@ -6661,6 +8372,11 @@
   dependencies:
     is-extglob "^2.1.1"
 
+is-iexplorer@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/is-iexplorer/-/is-iexplorer-1.0.0.tgz#1d72bc66d3fe22eaf6170dda8cf10943248cfc76"
+  integrity sha1-HXK8ZtP+Iur2Fw3ajPEJQySM/HY=
+
 is-installed-globally@^0.1.0:
   version "0.1.0"
   resolved "https://registry.yarnpkg.com/is-installed-globally/-/is-installed-globally-0.1.0.tgz#0dfd98f5a9111716dd535dda6492f67bf3d25a80"
@@ -6689,6 +8405,11 @@
   resolved "https://registry.yarnpkg.com/is-mobile/-/is-mobile-2.1.0.tgz#4c0cab72f3358dd9986007389b30729fae80da0b"
   integrity sha512-M5OhlZwh+aTlmRUvDg0Wq3uWVNa+w4DyZ2SjbrS+BhSLu9Po+JXHendC305ZEu+Hh7lywb19Zu4kYXu3L1Oo8A==
 
+is-mobile@^2.2.1:
+  version "2.2.1"
+  resolved "https://registry.yarnpkg.com/is-mobile/-/is-mobile-2.2.1.tgz#10f2320012c410cc285feecb13406bd586f1b2f8"
+  integrity sha512-6zELsfVFr326eq2CI53yvqq6YBanOxKBybwDT+MbMS2laBnK6Ez8m5XHSuTQQbnKRfpDzCod1CMWW5q3wZYMvA==
+
 is-negated-glob@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/is-negated-glob/-/is-negated-glob-1.0.0.tgz#6910bca5da8c95e784b5751b976cf5a10fee36d2"
@@ -6790,7 +8511,7 @@
     lowercase-keys "^1.0.0"
     proto-props "^2.0.0"
 
-is-regex@^1.0.4, is-regex@^1.0.5:
+is-regex@^1.0.4, is-regex@^1.0.5, is-regex@~1.0.5:
   version "1.0.5"
   resolved "https://registry.yarnpkg.com/is-regex/-/is-regex-1.0.5.tgz#39d589a358bf18967f726967120b8fc1aed74eae"
   integrity sha512-vlKW17SNq44owv5AQR3Cq0bQPEb8+kF3UKZ2fiZNOWtztYE5i0CzCZxFDwO58qAOWtxdBRVO/V5Qin1wjCqFYQ==
@@ -6824,11 +8545,21 @@
   resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-1.1.0.tgz#12d4a3dd4e68e0b79ceb8dbc84173ae80d91ca44"
   integrity sha1-EtSj3U5o4Lec6428hBc66A2RykQ=
 
+is-string-blank@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/is-string-blank/-/is-string-blank-1.0.1.tgz#866dca066d41d2894ebdfd2d8fe93e586e583a03"
+  integrity sha512-9H+ZBCVs3L9OYqv8nuUAzpcT9OTgMD1yAWrG7ihlnibdkbtB850heAmYWxHuXc4CHy4lKeK69tN+ny1K7gBIrw==
+
 is-string@^1.0.5:
   version "1.0.5"
   resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.5.tgz#40493ed198ef3ff477b8c7f92f644ec82a5cd3a6"
   integrity sha512-buY6VNRjhQMiF1qWDouloZlQbRhDPCebwxSjxMjxgemYT46YMd2NR0/H+fBhEfWX4A/w9TBJ+ol+okqJKFE6vQ==
 
+is-svg-path@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/is-svg-path/-/is-svg-path-1.0.2.tgz#77ab590c12b3d20348e5c7a13d0040c87784dda0"
+  integrity sha1-d6tZDBKz0gNI5cehPQBAyHeE3aA=
+
 is-svg@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/is-svg/-/is-svg-3.0.0.tgz#9321dbd29c212e5ca99c4fa9794c714bcafa2f75"
@@ -6887,6 +8618,11 @@
   resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11"
   integrity sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE=
 
+isarray@^2.0.1:
+  version "2.0.5"
+  resolved "https://registry.yarnpkg.com/isarray/-/isarray-2.0.5.tgz#8af1e4c1221244cc62459faf38940d4e644a5723"
+  integrity sha512-xHjhDr3cNBK0BzdUJSPXZntQUx/mwMS5Rw4A7lPJ90XGAO6ISP/ePDNuo0vhqOZU+UD5JoodwCAAoZQd3FeAKw==
+
 isexe@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10"
@@ -7600,6 +9336,11 @@
     array-includes "^3.0.3"
     object.assign "^4.1.0"
 
+kdbush@^3.0.0:
+  version "3.0.0"
+  resolved "https://registry.yarnpkg.com/kdbush/-/kdbush-3.0.0.tgz#f8484794d47004cc2d85ed3a79353dbe0abc2bf0"
+  integrity sha512-hRkd6/XW4HTsA9vjVpY9tuXJYLSlelnkTmVFu4M9/7MIYQtFcHpbugAU7UbOfjOiVSVYl2fqgBuJ32JUmRo5Ew==
+
 keyv@^3.0.0:
   version "3.1.0"
   resolved "https://registry.yarnpkg.com/keyv/-/keyv-3.1.0.tgz#ecc228486f69991e49e9476485a5be1e8fc5c4d9"
@@ -7685,6 +9426,11 @@
   resolved "https://registry.yarnpkg.com/left-pad/-/left-pad-1.3.0.tgz#5b8a3a7765dfe001261dde915589e782f8c94d1e"
   integrity sha512-XI5MPzVNApjAyhQzphX8BkmKsKUxD4LdyK24iZeQGinBN9yTQT3bFlCBy/aVx2HrNcqQGsdot8ghrjyrvMCoEA==
 
+lerp@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/lerp/-/lerp-1.0.3.tgz#a18c8968f917896de15ccfcc28d55a6b731e776e"
+  integrity sha1-oYyJaPkXiW3hXM/MKNVaa3Med24=
+
 less-loader@^5.0.0:
   version "5.0.0"
   resolved "https://registry.yarnpkg.com/less-loader/-/less-loader-5.0.0.tgz#498dde3a6c6c4f887458ee9ed3f086a12ad1b466"
@@ -7898,6 +9644,11 @@
   resolved "https://registry.yarnpkg.com/loglevel/-/loglevel-1.6.8.tgz#8a25fb75d092230ecd4457270d80b54e28011171"
   integrity sha512-bsU7+gc9AJ2SqpzxwU3+1fedl8zAntbtC5XYlt3s2j1hJcn2PsXSmgN8TaLG/J1/2mod4+cE/3vNL70/c1RNCA==
 
+longest@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/longest/-/longest-1.0.1.tgz#30a0b2da38f73770e8294a0d22e6625ed77d0097"
+  integrity sha1-MKCy2jj3N3DoKUoNIuZiXtd9AJc=
+
 loose-envify@^1.0.0, loose-envify@^1.1.0, loose-envify@^1.2.0, loose-envify@^1.3.1, loose-envify@^1.4.0:
   version "1.4.0"
   resolved "https://registry.yarnpkg.com/loose-envify/-/loose-envify-1.4.0.tgz#71ee51fa7be4caec1a63839f7e682d8132d30caf"
@@ -7956,6 +9707,13 @@
   dependencies:
     yallist "^3.0.2"
 
+magic-string@^0.25.3:
+  version "0.25.7"
+  resolved "https://registry.yarnpkg.com/magic-string/-/magic-string-0.25.7.tgz#3f497d6fd34c669c6798dcb821f2ef31f5445051"
+  integrity sha512-4CrMT5DOHTDk4HYDlzmwu4FVCcIYI8gauveasrdCu2IKIFOJ3f0v/8MDGJCDL9oD2ppz/Av1b0Nj345H9M+XIA==
+  dependencies:
+    sourcemap-codec "^1.4.4"
+
 make-dir@^1.0.0:
   version "1.3.0"
   resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-1.3.0.tgz#79c1033b80515bd6d24ec9933e860ca75ee27f0c"
@@ -8002,6 +9760,13 @@
   resolved "https://registry.yarnpkg.com/map-cache/-/map-cache-0.2.2.tgz#c32abd0bd6525d9b051645bb4f26ac5dc98a0dbf"
   integrity sha1-wyq9C9ZSXZsFFkW7TyasXcmKDb8=
 
+map-limit@0.0.1:
+  version "0.0.1"
+  resolved "https://registry.yarnpkg.com/map-limit/-/map-limit-0.0.1.tgz#eb7961031c0f0e8d001bf2d56fab685d58822f38"
+  integrity sha1-63lhAxwPDo0AG/LVb6toXViCLzg=
+  dependencies:
+    once "~1.3.0"
+
 map-obj@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/map-obj/-/map-obj-1.0.1.tgz#d933ceb9205d82bdcf4886f6742bdc2b4dea146d"
@@ -8019,6 +9784,83 @@
   dependencies:
     object-visit "^1.0.0"
 
+mapbox-gl@1.10.0:
+  version "1.10.0"
+  resolved "https://registry.yarnpkg.com/mapbox-gl/-/mapbox-gl-1.10.0.tgz#c33e74d1f328e820e245ff8ed7b5dbbbc4be204f"
+  integrity sha512-SrJXcR9s5yEsPuW2kKKumA1KqYW9RrL8j7ZcIh6glRQ/x3lwNMfwz/UEJAJcVNgeX+fiwzuBoDIdeGB/vSkZLQ==
+  dependencies:
+    "@mapbox/geojson-rewind" "^0.5.0"
+    "@mapbox/geojson-types" "^1.0.2"
+    "@mapbox/jsonlint-lines-primitives" "^2.0.2"
+    "@mapbox/mapbox-gl-supported" "^1.5.0"
+    "@mapbox/point-geometry" "^0.1.0"
+    "@mapbox/tiny-sdf" "^1.1.1"
+    "@mapbox/unitbezier" "^0.0.0"
+    "@mapbox/vector-tile" "^1.3.1"
+    "@mapbox/whoots-js" "^3.1.0"
+    csscolorparser "~1.0.3"
+    earcut "^2.2.2"
+    geojson-vt "^3.2.1"
+    gl-matrix "^3.2.1"
+    grid-index "^1.1.0"
+    minimist "^1.2.5"
+    murmurhash-js "^1.0.0"
+    pbf "^3.2.1"
+    potpack "^1.0.1"
+    quickselect "^2.0.0"
+    rw "^1.3.3"
+    supercluster "^7.0.0"
+    tinyqueue "^2.0.3"
+    vt-pbf "^3.1.1"
+
+marching-simplex-table@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/marching-simplex-table/-/marching-simplex-table-1.0.0.tgz#bc16256e0f8f9b558aa9b2872f8832d9433f52ea"
+  integrity sha1-vBYlbg+Pm1WKqbKHL4gy2UM/Uuo=
+  dependencies:
+    convex-hull "^1.0.3"
+
+mat4-decompose@^1.0.3:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/mat4-decompose/-/mat4-decompose-1.0.4.tgz#65eb4fe39d70878f7a444eb4624d52f7e7eb2faf"
+  integrity sha1-ZetP451wh496RE60Yk1S9+frL68=
+  dependencies:
+    gl-mat4 "^1.0.1"
+    gl-vec3 "^1.0.2"
+
+mat4-interpolate@^1.0.3:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/mat4-interpolate/-/mat4-interpolate-1.0.4.tgz#55ffe9eb3c35295e2c0d5a9f7725d9068a89ff74"
+  integrity sha1-Vf/p6zw1KV4sDVqfdyXZBoqJ/3Q=
+  dependencies:
+    gl-mat4 "^1.0.1"
+    gl-vec3 "^1.0.2"
+    mat4-decompose "^1.0.3"
+    mat4-recompose "^1.0.3"
+    quat-slerp "^1.0.0"
+
+mat4-recompose@^1.0.3:
+  version "1.0.4"
+  resolved "https://registry.yarnpkg.com/mat4-recompose/-/mat4-recompose-1.0.4.tgz#3953c230ff2473dc772ee014a52c925cf81b0e4d"
+  integrity sha1-OVPCMP8kc9x3LuAUpSySXPgbDk0=
+  dependencies:
+    gl-mat4 "^1.0.1"
+
+math-log2@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/math-log2/-/math-log2-1.0.1.tgz#fb8941be5f5ebe8979e718e6273b178e58694565"
+  integrity sha1-+4lBvl9evol55xjmJzsXjlhpRWU=
+
+matrix-camera-controller@^2.1.1, matrix-camera-controller@^2.1.3:
+  version "2.1.3"
+  resolved "https://registry.yarnpkg.com/matrix-camera-controller/-/matrix-camera-controller-2.1.3.tgz#35e5260cc1cd550962ba799f2d8d4e94b1a39370"
+  integrity sha1-NeUmDMHNVQliunmfLY1OlLGjk3A=
+  dependencies:
+    binary-search-bounds "^1.0.0"
+    gl-mat4 "^1.1.2"
+    gl-vec3 "^1.0.3"
+    mat4-interpolate "^1.0.3"
+
 md5.js@^1.3.4:
   version "1.3.5"
   resolved "https://registry.yarnpkg.com/md5.js/-/md5.js-1.3.5.tgz#b5d07b8e3216e3e27cd728d72f70d1e6a342005f"
@@ -8047,6 +9889,11 @@
     mimic-fn "^2.0.0"
     p-is-promise "^2.0.0"
 
+memoize-one@^5.0.0:
+  version "5.1.1"
+  resolved "https://registry.yarnpkg.com/memoize-one/-/memoize-one-5.1.1.tgz#047b6e3199b508eaec03504de71229b8eb1d75c0"
+  integrity sha512-HKeeBpWvqiVJD57ZUAsJNm71eHTykffzcLZVYWiVfQeI1rJtuEaS7hQiEpWfVVk18donPwJEcFKIkCmPJNOhHA==
+
 memory-fs@^0.2.0:
   version "0.2.0"
   resolved "https://registry.yarnpkg.com/memory-fs/-/memory-fs-0.2.0.tgz#f2bb25368bc121e391c2520de92969caee0a0290"
@@ -8268,7 +10115,7 @@
   resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.0.tgz#a35008b20f41383eec1fb914f4cd5df79a264284"
   integrity sha1-o1AIsg9BOD7sH7kU9M1d95omQoQ=
 
-minimist@^1.2.5:
+minimist@^1.2.5, minimist@~1.2.5:
   version "1.2.5"
   resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.5.tgz#67d66014b66a6a8aaa0c083c5fd58df4e4e97602"
   integrity sha512-FM9nNUYrRBAELZQT3xeZQ7fmMOBg6nWNmJKTcgsJeaLstP/UODVpGsr5OhXhhXg6f+qtJ8uiZ+PUxkDWcgIXLw==
@@ -8357,6 +10204,13 @@
   resolved "https://registry.yarnpkg.com/moment/-/moment-2.24.0.tgz#0d055d53f5052aa653c9f6eb68bb5d12bf5c2b5b"
   integrity sha512-bV7f+6l2QigeBBZSM/6yTNq4P2fNpSWj/0e7jQcy87A8e7o2nAfP/34/2ky5Vw4B9S446EtIhodAzkFCcR4dQg==
 
+monotone-convex-hull-2d@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/monotone-convex-hull-2d/-/monotone-convex-hull-2d-1.0.1.tgz#47f5daeadf3c4afd37764baa1aa8787a40eee08c"
+  integrity sha1-R/Xa6t88Sv03dkuqGqh4ekDu4Iw=
+  dependencies:
+    robust-orientation "^1.1.3"
+
 morgan@^1.9.1:
   version "1.9.1"
   resolved "https://registry.yarnpkg.com/morgan/-/morgan-1.9.1.tgz#0a8d16734a1d9afbc824b99df87e738e58e2da59"
@@ -8368,6 +10222,32 @@
     on-finished "~2.3.0"
     on-headers "~1.0.1"
 
+mouse-change@^1.4.0:
+  version "1.4.0"
+  resolved "https://registry.yarnpkg.com/mouse-change/-/mouse-change-1.4.0.tgz#c2b77e5bfa34a43ce1445c8157a4e4dc9895c14f"
+  integrity sha1-wrd+W/o0pDzhRFyBV6Tk3JiVwU8=
+  dependencies:
+    mouse-event "^1.0.0"
+
+mouse-event-offset@^3.0.2:
+  version "3.0.2"
+  resolved "https://registry.yarnpkg.com/mouse-event-offset/-/mouse-event-offset-3.0.2.tgz#dfd86a6e248c6ba8cad53b905d5037a2063e9984"
+  integrity sha1-39hqbiSMa6jK1TuQXVA3ogY+mYQ=
+
+mouse-event@^1.0.0:
+  version "1.0.5"
+  resolved "https://registry.yarnpkg.com/mouse-event/-/mouse-event-1.0.5.tgz#b3789edb7109997d5a932d1d01daa1543a501732"
+  integrity sha1-s3ie23EJmX1aky0dAdqhVDpQFzI=
+
+mouse-wheel@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/mouse-wheel/-/mouse-wheel-1.2.0.tgz#6d2903b1ea8fb48e61f1b53b9036773f042cdb5c"
+  integrity sha1-bSkDseqPtI5h8bU7kDZ3PwQs21w=
+  dependencies:
+    right-now "^1.0.0"
+    signum "^1.0.0"
+    to-px "^1.0.1"
+
 move-concurrently@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/move-concurrently/-/move-concurrently-1.0.1.tgz#be2c005fda32e0b29af1f05d7c4b33214c701f92"
@@ -8413,6 +10293,18 @@
   resolved "https://registry.yarnpkg.com/multimap/-/multimap-1.1.0.tgz#5263febc085a1791c33b59bb3afc6a76a2a10ca8"
   integrity sha512-0ZIR9PasPxGXmRsEF8jsDzndzHDj7tIav+JUmvIFB/WHswliFnquxECT/De7GR4yg99ky/NlRKJT82G1y271bw==
 
+mumath@^3.3.4:
+  version "3.3.4"
+  resolved "https://registry.yarnpkg.com/mumath/-/mumath-3.3.4.tgz#48d4a0f0fd8cad4e7b32096ee89b161a63d30bbf"
+  integrity sha1-SNSg8P2MrU57Mglu6JsWGmPTC78=
+  dependencies:
+    almost-equal "^1.1.0"
+
+murmurhash-js@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/murmurhash-js/-/murmurhash-js-1.0.0.tgz#b06278e21fc6c37fa5313732b0412bcb6ae15f51"
+  integrity sha1-sGJ44h/Gw3+lMTcysEEry2rhX1E=
+
 mutationobserver-shim@^0.3.2:
   version "0.3.3"
   resolved "https://registry.yarnpkg.com/mutationobserver-shim/-/mutationobserver-shim-0.3.3.tgz#65869630bc89d7bf8c9cd9cb82188cd955aacd2b"
@@ -8455,6 +10347,96 @@
   resolved "https://registry.yarnpkg.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7"
   integrity sha1-Sr6/7tdUHywnrPspvbvRXI1bpPc=
 
+ndarray-extract-contour@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/ndarray-extract-contour/-/ndarray-extract-contour-1.0.1.tgz#0aee113a3a33b226b90c4888cf877bf4751305e4"
+  integrity sha1-Cu4ROjozsia5DEiIz4d79HUTBeQ=
+  dependencies:
+    typedarray-pool "^1.0.0"
+
+ndarray-fill@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/ndarray-fill/-/ndarray-fill-1.0.2.tgz#a30a60f7188e0c9582fcdd58896acdcb522a1ed6"
+  integrity sha1-owpg9xiODJWC/N1YiWrNy1IqHtY=
+  dependencies:
+    cwise "^1.0.10"
+
+ndarray-gradient@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/ndarray-gradient/-/ndarray-gradient-1.0.0.tgz#b7491a515c6a649f19a62324fff6f27fc8c25393"
+  integrity sha1-t0kaUVxqZJ8ZpiMk//byf8jCU5M=
+  dependencies:
+    cwise-compiler "^1.0.0"
+    dup "^1.0.0"
+
+ndarray-homography@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/ndarray-homography/-/ndarray-homography-1.0.0.tgz#c35516ea86bc2862b4e804a236a2707309fe296b"
+  integrity sha1-w1UW6oa8KGK06ASiNqJwcwn+KWs=
+  dependencies:
+    gl-matrix-invert "^1.0.0"
+    ndarray-warp "^1.0.0"
+
+ndarray-linear-interpolate@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/ndarray-linear-interpolate/-/ndarray-linear-interpolate-1.0.0.tgz#78bc92b85b9abc15b6e67ee65828f9e2137ae72b"
+  integrity sha1-eLySuFuavBW25n7mWCj54hN65ys=
+
+ndarray-ops@^1.1.0, ndarray-ops@^1.2.1, ndarray-ops@^1.2.2:
+  version "1.2.2"
+  resolved "https://registry.yarnpkg.com/ndarray-ops/-/ndarray-ops-1.2.2.tgz#59e88d2c32a7eebcb1bc690fae141579557a614e"
+  integrity sha1-WeiNLDKn7ryxvGkPrhQVeVV6YU4=
+  dependencies:
+    cwise-compiler "^1.0.0"
+
+ndarray-pack@^1.2.1:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/ndarray-pack/-/ndarray-pack-1.2.1.tgz#8caebeaaa24d5ecf70ff86020637977da8ee585a"
+  integrity sha1-jK6+qqJNXs9w/4YCBjeXfajuWFo=
+  dependencies:
+    cwise-compiler "^1.1.2"
+    ndarray "^1.0.13"
+
+ndarray-scratch@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/ndarray-scratch/-/ndarray-scratch-1.2.0.tgz#6304636d62eba93db4727ac13c693341dba50e01"
+  integrity sha1-YwRjbWLrqT20cnrBPGkzQdulDgE=
+  dependencies:
+    ndarray "^1.0.14"
+    ndarray-ops "^1.2.1"
+    typedarray-pool "^1.0.2"
+
+ndarray-sort@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/ndarray-sort/-/ndarray-sort-1.0.1.tgz#fea05b4cb834c7f4e0216a354f3ca751300dfd6a"
+  integrity sha1-/qBbTLg0x/TgIWo1TzynUTAN/Wo=
+  dependencies:
+    typedarray-pool "^1.0.0"
+
+ndarray-warp@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/ndarray-warp/-/ndarray-warp-1.0.1.tgz#a8a125aaabba0bebf93bd6ca83e6abd6822a34e0"
+  integrity sha1-qKElqqu6C+v5O9bKg+ar1oIqNOA=
+  dependencies:
+    cwise "^1.0.4"
+    ndarray-linear-interpolate "^1.0.0"
+
+ndarray@1.0.18:
+  version "1.0.18"
+  resolved "https://registry.yarnpkg.com/ndarray/-/ndarray-1.0.18.tgz#b60d3a73224ec555d0faa79711e502448fd3f793"
+  integrity sha1-tg06cyJOxVXQ+qeXEeUCRI/T95M=
+  dependencies:
+    iota-array "^1.0.0"
+    is-buffer "^1.0.2"
+
+ndarray@^1.0.11, ndarray@^1.0.13, ndarray@^1.0.14, ndarray@^1.0.15, ndarray@^1.0.18:
+  version "1.0.19"
+  resolved "https://registry.yarnpkg.com/ndarray/-/ndarray-1.0.19.tgz#6785b5f5dfa58b83e31ae5b2a058cfd1ab3f694e"
+  integrity sha512-B4JHA4vdyZU30ELBw3g7/p9bZupyew5a7tX1Y/gGeF2hafrPaQZhgrGQfsvgfYbgdFZjYwuEcnaobeM/WMW+HQ==
+  dependencies:
+    iota-array "^1.0.0"
+    is-buffer "^1.0.2"
+
 negotiator@0.6.2:
   version "0.6.2"
   resolved "https://registry.yarnpkg.com/negotiator/-/negotiator-0.6.2.tgz#feacf7ccf525a77ae9634436a64883ffeca346fb"
@@ -8470,6 +10452,13 @@
   resolved "https://registry.yarnpkg.com/next-tick/-/next-tick-1.0.0.tgz#ca86d1fe8828169b0120208e3dc8424b9db8342c"
   integrity sha1-yobR/ogoFpsBICCOPchCS524NCw=
 
+nextafter@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/nextafter/-/nextafter-1.0.0.tgz#b7d77b535310e3e097e6025abb0a903477ec1a3a"
+  integrity sha1-t9d7U1MQ4+CX5gJauwqQNHfsGjo=
+  dependencies:
+    double-bits "^1.1.0"
+
 nice-try@^1.0.4:
   version "1.0.5"
   resolved "https://registry.yarnpkg.com/nice-try/-/nice-try-1.0.5.tgz#a3378a7696ce7d223e88fc9b764bd7ef1089e366"
@@ -8585,6 +10574,18 @@
   resolved "https://registry.yarnpkg.com/normalize-range/-/normalize-range-0.1.2.tgz#2d10c06bdfd312ea9777695a4d28439456b75942"
   integrity sha1-LRDAa9/TEuqXd2laTShDlFa3WUI=
 
+normalize-svg-path@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/normalize-svg-path/-/normalize-svg-path-1.0.1.tgz#6f729ad6b70bb4ca4eff2fe4b107489efe1d56fe"
+  integrity sha1-b3Ka1rcLtMpO/y/ksQdInv4dVv4=
+  dependencies:
+    svg-arc-to-cubic-bezier "^3.0.0"
+
+normalize-svg-path@~0.1.0:
+  version "0.1.0"
+  resolved "https://registry.yarnpkg.com/normalize-svg-path/-/normalize-svg-path-0.1.0.tgz#456360e60ece75fbef7b5d7e160480e7ffd16fe5"
+  integrity sha1-RWNg5g7Odfvve11+FgSA5//Rb+U=
+
 normalize-url@1.9.1:
   version "1.9.1"
   resolved "https://registry.yarnpkg.com/normalize-url/-/normalize-url-1.9.1.tgz#2cc0d66b31ea23036458436e3620d85954c66c3c"
@@ -8605,6 +10606,11 @@
   resolved "https://registry.yarnpkg.com/normalize-url/-/normalize-url-4.5.0.tgz#453354087e6ca96957bd8f5baf753f5982142129"
   integrity sha512-2s47yzUxdexf1OhyRi4Em83iQk0aPvwTddtFz4hnSSw9dCEsLEGf6SwIO8ss/19S9iBb5sJaOuTvTGDeZI00BQ==
 
+normals@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/normals/-/normals-1.1.0.tgz#325b595ed34afe467a6c55a14fd9085787ff59c0"
+  integrity sha1-MltZXtNK/kZ6bFWhT9kIV4f/WcA=
+
 npm-run-all@^4.1.5:
   version "4.1.5"
   resolved "https://registry.yarnpkg.com/npm-run-all/-/npm-run-all-4.1.5.tgz#04476202a15ee0e2e214080861bff12a51d98fba"
@@ -8639,11 +10645,23 @@
   resolved "https://registry.yarnpkg.com/num2fraction/-/num2fraction-1.2.2.tgz#6f682b6a027a4e9ddfa4564cd2589d1d4e669ede"
   integrity sha1-b2gragJ6Tp3fpFZM0lidHU5mnt4=
 
+number-is-integer@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/number-is-integer/-/number-is-integer-1.0.1.tgz#e59bca172ffed27318e79c7ceb6cb72c095b2152"
+  integrity sha1-5ZvKFy/+0nMY55x862y3LAlbIVI=
+  dependencies:
+    is-finite "^1.0.1"
+
 number-is-nan@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/number-is-nan/-/number-is-nan-1.0.1.tgz#097b602b53422a522c1afb8790318336941a011d"
   integrity sha1-CXtgK1NCKlIsGvuHkDGDNpQaAR0=
 
+numeric@^1.2.6:
+  version "1.2.6"
+  resolved "https://registry.yarnpkg.com/numeric/-/numeric-1.2.6.tgz#765b02bef97988fcf880d4eb3f36b80fa31335aa"
+  integrity sha1-dlsCvvl5iPz4gNTrPza4D6MTNao=
+
 nwsapi@^2.0.7, nwsapi@^2.1.3:
   version "2.2.0"
   resolved "https://registry.yarnpkg.com/nwsapi/-/nwsapi-2.2.0.tgz#204879a9e3d068ff2a55139c2c772780681a38b7"
@@ -8678,21 +10696,31 @@
   resolved "https://registry.yarnpkg.com/object-hash/-/object-hash-2.0.3.tgz#d12db044e03cd2ca3d77c0570d87225b02e1e6ea"
   integrity sha512-JPKn0GMu+Fa3zt3Bmr66JhokJU5BaNBIh4ZeTlaCBzrBsOeXzwcKKAK1tbLiPKgvwmPXsDvvLHoWh5Bm7ofIYg==
 
-object-inspect@^1.7.0:
+object-inspect@^1.7.0, object-inspect@~1.7.0:
   version "1.7.0"
   resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.7.0.tgz#f4f6bd181ad77f006b5ece60bd0b6f398ff74a67"
   integrity sha512-a7pEHdh1xKIAgTySUGgLMx/xwDZskN1Ud6egYYN3EdRW4ZMPNEDUTF+hwy2LUC+Bl+SyLXANnwz/jyh/qutKUw==
 
+object-inspect@~0.4.0:
+  version "0.4.0"
+  resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-0.4.0.tgz#f5157c116c1455b243b06ee97703392c5ad89fec"
+  integrity sha1-9RV8EWwUVbJDsG7pdwM5LFrYn+w=
+
 object-is@^1.0.1:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/object-is/-/object-is-1.0.2.tgz#6b80eb84fe451498f65007982f035a5b445edec4"
   integrity sha512-Epah+btZd5wrrfjkJZq1AOB9O6OxUQto45hzFd7lXGrpHPGE0W1k+426yrZV+k6NJOzLNNW/nVsmZdIWsAqoOQ==
 
-object-keys@^1.0.11, object-keys@^1.0.12, object-keys@^1.1.1:
+object-keys@^1.0.11, object-keys@^1.0.12, object-keys@^1.0.6, object-keys@^1.0.9, object-keys@^1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-1.1.1.tgz#1c47f272df277f3b1daf061677d9c82e2322c60e"
   integrity sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==
 
+object-keys@~0.4.0:
+  version "0.4.0"
+  resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-0.4.0.tgz#28a6aae7428dd2c3a92f3d95f21335dd204e0336"
+  integrity sha1-KKaq50KN0sOpLz2V8hM13SBOAzY=
+
 object-path@0.11.4:
   version "0.11.4"
   resolved "https://registry.yarnpkg.com/object-path/-/object-path-0.11.4.tgz#370ae752fbf37de3ea70a861c23bba8915691949"
@@ -8791,6 +10819,13 @@
   dependencies:
     wrappy "1"
 
+once@~1.3.0:
+  version "1.3.3"
+  resolved "https://registry.yarnpkg.com/once/-/once-1.3.3.tgz#b2e261557ce4c314ec8304f3fa82663e4297ca20"
+  integrity sha1-suJhVXzkwxTsgwTz+oJmPkKXyiA=
+  dependencies:
+    wrappy "1"
+
 onetime@^5.1.0:
   version "5.1.0"
   resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.0.tgz#fff0f3c91617fe62bb50189636e99ac8a6df7be5"
@@ -8857,6 +10892,14 @@
     type-check "~0.3.2"
     word-wrap "~1.2.3"
 
+orbit-camera-controller@^4.0.0:
+  version "4.0.0"
+  resolved "https://registry.yarnpkg.com/orbit-camera-controller/-/orbit-camera-controller-4.0.0.tgz#6e2b36f0e7878663c330f50da9b7ce686c277005"
+  integrity sha1-bis28OeHhmPDMPUNqbfOaGwncAU=
+  dependencies:
+    filtered-vector "^1.2.1"
+    gl-mat4 "^1.0.3"
+
 original@^1.0.0:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/original/-/original-1.0.2.tgz#e442a61cffe1c5fd20a65f3261c26663b303f25f"
@@ -8869,6 +10912,11 @@
   resolved "https://registry.yarnpkg.com/os-browserify/-/os-browserify-0.3.0.tgz#854373c7f5c2315914fc9bfc6bd8238fdda1ec27"
   integrity sha1-hUNzx/XCMVkU/Jv8a9gjj92h7Cc=
 
+os-homedir@^2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/os-homedir/-/os-homedir-2.0.0.tgz#a0c76bb001a8392a503cbd46e7e650b3423a923c"
+  integrity sha512-saRNz0DSC5C/I++gFIaJTXoFJMRwiP5zHar5vV3xQ2TkgEw6hDCcU5F272JjUylpiVgBrZNQHnfjkLabTfb92Q==
+
 os-locale@^3.0.0:
   version "3.1.0"
   resolved "https://registry.yarnpkg.com/os-locale/-/os-locale-3.1.0.tgz#a802a6ee17f24c10483ab9935719cef4ed16bf1a"
@@ -9001,6 +11049,13 @@
     registry-url "^5.0.0"
     semver "^6.2.0"
 
+pad-left@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/pad-left/-/pad-left-1.0.2.tgz#19e5735ea98395a26cedc6ab926ead10f3100d4c"
+  integrity sha1-GeVzXqmDlaJs7carkm6tEPMQDUw=
+  dependencies:
+    repeat-string "^1.3.0"
+
 pako@~1.0.5:
   version "1.0.10"
   resolved "https://registry.yarnpkg.com/pako/-/pako-1.0.10.tgz#4328badb5086a426aa90f541977d4955da5c9732"
@@ -9030,6 +11085,11 @@
   dependencies:
     callsites "^3.0.0"
 
+parenthesis@^3.1.5:
+  version "3.1.7"
+  resolved "https://registry.yarnpkg.com/parenthesis/-/parenthesis-3.1.7.tgz#01c89b603a2a6a262ec47554e74ed154a9be2aa6"
+  integrity sha512-iMtu+HCbLXVrpf6Ys/4YKhcFxbux3xK4ZVB9r+a2kMSqeeQWQoDNYlXIsOjwlT2ldYXZ3k5PVeBnYn7fbAo/Bg==
+
 parse-asn1@^5.0.0:
   version "5.1.5"
   resolved "https://registry.yarnpkg.com/parse-asn1/-/parse-asn1-5.1.5.tgz#003271343da58dc94cace494faef3d2147ecea0e"
@@ -9072,6 +11132,23 @@
   resolved "https://registry.yarnpkg.com/parse-ms/-/parse-ms-2.1.0.tgz#348565a753d4391fa524029956b172cb7753097d"
   integrity sha512-kHt7kzLoS9VBZfUsiKjv43mr91ea+U05EyKkEtqp7vNbHxmaVuEqN7XxeEVnGrMtYOAxGrDElSi96K7EgO1zCA==
 
+parse-rect@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/parse-rect/-/parse-rect-1.2.0.tgz#e0a5b0dbaaaee637a0a1eb9779969e19399d8dec"
+  integrity sha512-4QZ6KYbnE6RTwg9E0HpLchUM9EZt6DnDxajFZZDSV4p/12ZJEvPO702DZpGvRYEPo00yKDys7jASi+/w7aO8LA==
+  dependencies:
+    pick-by-alias "^1.2.0"
+
+parse-svg-path@^0.1.2:
+  version "0.1.2"
+  resolved "https://registry.yarnpkg.com/parse-svg-path/-/parse-svg-path-0.1.2.tgz#7a7ec0d1eb06fa5325c7d3e009b859a09b5d49eb"
+  integrity sha1-en7A0esG+lMlx9PgCbhZoJtdSes=
+
+parse-unit@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/parse-unit/-/parse-unit-1.0.1.tgz#7e1bb6d5bef3874c28e392526a2541170291eecf"
+  integrity sha1-fhu21b7zh0wo45JSaiVBFwKR7s8=
+
 parse5@4.0.0:
   version "4.0.0"
   resolved "https://registry.yarnpkg.com/parse5/-/parse5-4.0.0.tgz#6d78656e3da8d78b4ec0b906f7c08ef1dfe3f608"
@@ -9183,6 +11260,14 @@
   resolved "https://registry.yarnpkg.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b"
   integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==
 
+pbf@^3.0.5, pbf@^3.2.1:
+  version "3.2.1"
+  resolved "https://registry.yarnpkg.com/pbf/-/pbf-3.2.1.tgz#b4c1b9e72af966cd82c6531691115cc0409ffe2a"
+  integrity sha512-ClrV7pNOn7rtmoQVF4TS1vyU0WhYRnP92fzbfF75jAIwpnzdJXf8iTd4CMEqO4yUenH6NDqLiwjqlh6QgZzgLQ==
+  dependencies:
+    ieee754 "^1.1.12"
+    resolve-protobuf-schema "^2.1.0"
+
 pbkdf2@^3.0.3:
   version "3.0.17"
   resolved "https://registry.yarnpkg.com/pbkdf2/-/pbkdf2-3.0.17.tgz#976c206530617b14ebb32114239f7b09336e93a6"
@@ -9199,6 +11284,26 @@
   resolved "https://registry.yarnpkg.com/performance-now/-/performance-now-2.1.0.tgz#6309f4e0e5fa913ec1c69307ae364b4b377c9e7b"
   integrity sha1-Ywn04OX6kT7BxpMHrjZLSzd8nns=
 
+permutation-parity@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/permutation-parity/-/permutation-parity-1.0.0.tgz#0174d51fca704b11b9a4b152b23d537fdc6b5ef4"
+  integrity sha1-AXTVH8pwSxG5pLFSsj1Tf9xrXvQ=
+  dependencies:
+    typedarray-pool "^1.0.0"
+
+permutation-rank@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/permutation-rank/-/permutation-rank-1.0.0.tgz#9fd98bbcecf08fbf5994b5eadc94a62e679483b5"
+  integrity sha1-n9mLvOzwj79ZlLXq3JSmLmeUg7U=
+  dependencies:
+    invert-permutation "^1.0.0"
+    typedarray-pool "^1.0.0"
+
+pick-by-alias@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/pick-by-alias/-/pick-by-alias-1.2.0.tgz#5f7cb2b1f21a6e1e884a0c87855aa4a37361107b"
+  integrity sha1-X3yysfIabh6ISgyHhVqko3NhEHs=
+
 picomatch@^2.0.4, picomatch@^2.0.5, picomatch@^2.2.1:
   version "2.2.2"
   resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.2.2.tgz#21f333e9b6b8eaff02468f5146ea406d345f4dad"
@@ -9285,6 +11390,27 @@
   dependencies:
     find-up "^2.1.0"
 
+planar-dual@^1.0.0:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/planar-dual/-/planar-dual-1.0.2.tgz#b6a4235523b1b0cb79e5f926f8ea335dd982d563"
+  integrity sha1-tqQjVSOxsMt55fkm+OozXdmC1WM=
+  dependencies:
+    compare-angle "^1.0.0"
+    dup "^1.0.0"
+
+planar-graph-to-polyline@^1.0.0:
+  version "1.0.5"
+  resolved "https://registry.yarnpkg.com/planar-graph-to-polyline/-/planar-graph-to-polyline-1.0.5.tgz#882b8605199ba88bfd464c9553303556c52b988a"
+  integrity sha1-iCuGBRmbqIv9RkyVUzA1VsUrmIo=
+  dependencies:
+    edges-to-adjacency-list "^1.0.0"
+    planar-dual "^1.0.0"
+    point-in-big-polygon "^2.0.0"
+    robust-orientation "^1.0.1"
+    robust-sum "^1.0.0"
+    two-product "^1.0.0"
+    uniq "^1.0.0"
+
 please-upgrade-node@^3.2.0:
   version "3.2.0"
   resolved "https://registry.yarnpkg.com/please-upgrade-node/-/please-upgrade-node-3.2.0.tgz#aeddd3f994c933e4ad98b99d9a556efa0e2fe942"
@@ -9292,6 +11418,77 @@
   dependencies:
     semver-compare "^1.0.0"
 
+plotly.js@^1.54.1:
+  version "1.54.1"
+  resolved "https://registry.yarnpkg.com/plotly.js/-/plotly.js-1.54.1.tgz#7aed029e80b39c4b095957e837ee24a737b49b93"
+  integrity sha512-1tlsEQkUgX2BaQr3Eu+5drki3eyor5EhpolFRteuQouRFX9mrg9Pms39Lak29lbAx4W3ZQoCZm8DcKkQCOe9YQ==
+  dependencies:
+    "@plotly/d3-sankey" "0.7.2"
+    "@plotly/d3-sankey-circular" "0.33.1"
+    "@turf/area" "^6.0.1"
+    "@turf/bbox" "^6.0.1"
+    "@turf/centroid" "^6.0.2"
+    alpha-shape "^1.0.0"
+    canvas-fit "^1.5.0"
+    color-normalize "^1.5.0"
+    color-rgba "^2.1.1"
+    convex-hull "^1.0.3"
+    country-regex "^1.1.0"
+    d3 "^3.5.17"
+    d3-force "^1.2.1"
+    d3-hierarchy "^1.1.9"
+    d3-interpolate "^1.4.0"
+    delaunay-triangulate "^1.1.6"
+    es6-promise "^4.2.8"
+    fast-isnumeric "^1.1.4"
+    gl-cone3d "^1.5.2"
+    gl-contour2d "^1.1.7"
+    gl-error3d "^1.0.16"
+    gl-heatmap2d "^1.0.6"
+    gl-line3d "1.2.1"
+    gl-mat4 "^1.2.0"
+    gl-mesh3d "^2.3.1"
+    gl-plot2d "^1.4.4"
+    gl-plot3d "^2.4.5"
+    gl-pointcloud2d "^1.0.3"
+    gl-scatter3d "^1.2.3"
+    gl-select-box "^1.0.4"
+    gl-spikes2d "^1.0.2"
+    gl-streamtube3d "^1.4.1"
+    gl-surface3d "^1.5.2"
+    gl-text "^1.1.8"
+    glslify "^7.0.0"
+    has-hover "^1.0.1"
+    has-passive-events "^1.0.0"
+    is-mobile "^2.2.1"
+    mapbox-gl "1.10.0"
+    matrix-camera-controller "^2.1.3"
+    mouse-change "^1.4.0"
+    mouse-event-offset "^3.0.2"
+    mouse-wheel "^1.2.0"
+    ndarray "1.0.18"
+    ndarray-fill "^1.0.2"
+    ndarray-homography "^1.0.0"
+    parse-svg-path "^0.1.2"
+    point-cluster "^3.1.8"
+    polybooljs "^1.2.0"
+    regl "1.3.11"
+    regl-error2d "^2.0.8"
+    regl-line2d "^3.0.15"
+    regl-scatter2d "^3.1.8"
+    regl-splom "^1.0.8"
+    right-now "^1.0.0"
+    robust-orientation "^1.1.3"
+    sane-topojson "^4.0.0"
+    strongly-connected-components "^1.0.1"
+    superscript-text "^1.0.0"
+    svg-path-sdf "^1.1.3"
+    tinycolor2 "^1.4.1"
+    to-px "1.0.1"
+    topojson-client "^3.1.0"
+    webgl-context "^2.2.0"
+    world-calendars "^1.0.3"
+
 plur@^3.0.1:
   version "3.1.1"
   resolved "https://registry.yarnpkg.com/plur/-/plur-3.1.1.tgz#60267967866a8d811504fe58f2faaba237546a5b"
@@ -9316,6 +11513,46 @@
   dependencies:
     ts-pnp "^1.1.6"
 
+point-cluster@^3.1.8:
+  version "3.1.8"
+  resolved "https://registry.yarnpkg.com/point-cluster/-/point-cluster-3.1.8.tgz#a63625fd8964f2a5b446025a1acf8bcac42500c0"
+  integrity sha512-7klIr45dpMeZuqjIK9+qBg3m2IhyZJNJkdqjJFw0Olq75FM8ojrTMjClVUrMjNYRVqtwztxCHH71Fyjhg+YwyQ==
+  dependencies:
+    array-bounds "^1.0.1"
+    array-normalize "^1.1.4"
+    binary-search-bounds "^2.0.4"
+    bubleify "^1.1.0"
+    clamp "^1.0.1"
+    defined "^1.0.0"
+    dtype "^2.0.0"
+    flatten-vertex-data "^1.0.2"
+    is-obj "^1.0.1"
+    math-log2 "^1.0.1"
+    parse-rect "^1.2.0"
+    pick-by-alias "^1.2.0"
+
+point-in-big-polygon@^2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/point-in-big-polygon/-/point-in-big-polygon-2.0.0.tgz#39b613ea6cf17d6b43e188f77f34c44c6b33ba55"
+  integrity sha1-ObYT6mzxfWtD4Yj3fzTETGszulU=
+  dependencies:
+    binary-search-bounds "^1.0.0"
+    interval-tree-1d "^1.0.1"
+    robust-orientation "^1.1.3"
+    slab-decomposition "^1.0.1"
+
+polybooljs@^1.2.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/polybooljs/-/polybooljs-1.2.0.tgz#b4390c2e079d4c262d3b2504c6288d95ba7a4758"
+  integrity sha1-tDkMLgedTCYtOyUExiiNlbp6R1g=
+
+polytope-closest-point@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/polytope-closest-point/-/polytope-closest-point-1.0.0.tgz#e6e57f4081ab5e8c778b811ef06e2c48ae338c3f"
+  integrity sha1-5uV/QIGrXox3i4Ee8G4sSK4zjD8=
+  dependencies:
+    numeric "^1.2.6"
+
 portfinder@^1.0.25:
   version "1.0.26"
   resolved "https://registry.yarnpkg.com/portfinder/-/portfinder-1.0.26.tgz#475658d56ca30bed72ac7f1378ed350bd1b64e70"
@@ -10006,6 +12243,11 @@
     source-map "^0.6.1"
     supports-color "^6.1.0"
 
+potpack@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/potpack/-/potpack-1.0.1.tgz#d1b1afd89e4c8f7762865ec30bd112ab767e2ebf"
+  integrity sha512-15vItUAbViaYrmaB/Pbw7z6qX2xENbFSTA7Ii4tgbPtasxm5v6ryKhKtL91tpWovDJzTiZqdwzhcFBCwiMVdVw==
+
 prelude-ls@~1.1.2:
   version "1.1.2"
   resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.1.2.tgz#21932a549f5e52ffd9a827f570e04be62a97da54"
@@ -10033,7 +12275,7 @@
   resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.0.4.tgz#2d1bae173e355996ee355ec9830a7a1ee05457ef"
   integrity sha512-SVJIQ51spzFDvh4fIbCLvciiDMCrRhlN3mbZvv/+ycjvmF5E73bKdGfU8QDLNmjYJf+lsGnDBC4UUnvTe5OO0w==
 
-pretty-bytes@^5.1.0, pretty-bytes@^5.3.0:
+pretty-bytes@^5.1.0:
   version "5.3.0"
   resolved "https://registry.yarnpkg.com/pretty-bytes/-/pretty-bytes-5.3.0.tgz#f2849e27db79fb4d6cfe24764fc4134f165989f2"
   integrity sha512-hjGrh+P926p4R4WbaB6OckyRtO0F0/lQBiT+0gnxjV+5kjPBrfVBFCsCLbMqVQeydvIoouYTCmmEURiH3R1Bdg==
@@ -10124,6 +12366,11 @@
   resolved "https://registry.yarnpkg.com/proto-props/-/proto-props-2.0.0.tgz#8ac6e6dec658545815c623a3bc81580deda9a181"
   integrity sha512-2yma2tog9VaRZY2mn3Wq51uiSW4NcPYT1cQdBagwyrznrilKSZwIZ0UG3ZPL/mx+axEns0hE35T5ufOYZXEnBQ==
 
+protocol-buffers-schema@^3.3.1:
+  version "3.4.0"
+  resolved "https://registry.yarnpkg.com/protocol-buffers-schema/-/protocol-buffers-schema-3.4.0.tgz#2f0ea31ca96627d680bf2fefae7ebfa2b6453eae"
+  integrity sha512-G/2kcamPF2S49W5yaMGdIpkG6+5wZF0fzBteLKgEHjbNzqjZQ85aAs1iJGto31EJaSTkNvHs5IXuHSaTLWBAiA==
+
 proxy-addr@~2.0.5:
   version "2.0.5"
   resolved "https://registry.yarnpkg.com/proxy-addr/-/proxy-addr-2.0.5.tgz#34cbd64a2d81f4b1fd21e76f9f06c8a45299ee34"
@@ -10206,6 +12453,18 @@
   dependencies:
     escape-goat "^2.0.0"
 
+pxls@^2.0.0:
+  version "2.3.2"
+  resolved "https://registry.yarnpkg.com/pxls/-/pxls-2.3.2.tgz#79100d2cc95089fc6e00053a9d93c1ddddb2c7b4"
+  integrity sha512-pQkwgbLqWPcuES5iEmGa10OlCf5xG0blkIF3dg7PpRZShbTYcvAdfFfGL03SMrkaSUaa/V0UpN9HWg40O2AIIw==
+  dependencies:
+    arr-flatten "^1.1.0"
+    compute-dims "^1.1.0"
+    flip-pixels "^1.0.2"
+    is-browser "^2.1.0"
+    is-buffer "^2.0.3"
+    to-uint8 "^1.4.1"
+
 q@^1.1.2:
   version "1.5.1"
   resolved "https://registry.yarnpkg.com/q/-/q-1.5.1.tgz#7e32f75b41381291d04611f1bf14109ac00651d7"
@@ -10221,6 +12480,18 @@
   resolved "https://registry.yarnpkg.com/qs/-/qs-6.5.2.tgz#cb3ae806e8740444584ef154ce8ee98d403f3e36"
   integrity sha512-N5ZAX4/LxJmF+7wN74pUD6qAh9/wnvdQcjq9TZjevvXzSUo7bfmw91saqMjzGS2xq91/odN2dW/WOl7qQHNDGA==
 
+quantize@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/quantize/-/quantize-1.0.2.tgz#d25ac200a77b6d70f40127ca171a10e33c8546de"
+  integrity sha1-0lrCAKd7bXD0ASfKFxoQ4zyFRt4=
+
+quat-slerp@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/quat-slerp/-/quat-slerp-1.0.1.tgz#2baa15ce3a6bbdc3241d972eb17283139ed69f29"
+  integrity sha1-K6oVzjprvcMkHZcusXKDE57Wnyk=
+  dependencies:
+    gl-quat "^1.0.0"
+
 query-string@^4.1.0:
   version "4.3.4"
   resolved "https://registry.yarnpkg.com/query-string/-/query-string-4.3.4.tgz#bbb693b9ca915c232515b228b1a02b609043dbeb"
@@ -10249,6 +12520,19 @@
   resolved "https://registry.yarnpkg.com/quick-lru/-/quick-lru-1.1.0.tgz#4360b17c61136ad38078397ff11416e186dcfbb8"
   integrity sha1-Q2CxfGETatOAeDl/8RQW4Ybc+7g=
 
+quickselect@^2.0.0:
+  version "2.0.0"
+  resolved "https://registry.yarnpkg.com/quickselect/-/quickselect-2.0.0.tgz#f19680a486a5eefb581303e023e98faaf25dd018"
+  integrity sha512-RKJ22hX8mHe3Y6wH/N3wCM6BWtjaxIyyUIkpHOvfFnxdI4yD4tBXEBKSbriGujF6jnSVkJrffuo6vxACiSSxIw==
+
+quote-stream@~0.0.0:
+  version "0.0.0"
+  resolved "https://registry.yarnpkg.com/quote-stream/-/quote-stream-0.0.0.tgz#cde29e94c409b16e19dc7098b89b6658f9721d3b"
+  integrity sha1-zeKelMQJsW4Z3HCYuJtmWPlyHTs=
+  dependencies:
+    minimist "0.0.8"
+    through2 "~0.4.1"
+
 raf@^3.4.0, raf@^3.4.1:
   version "3.4.1"
   resolved "https://registry.yarnpkg.com/raf/-/raf-3.4.1.tgz#0742e99a4a6552f445d73e3ee0328af0ff1ede39"
@@ -10276,6 +12560,13 @@
   resolved "https://registry.yarnpkg.com/range-parser/-/range-parser-1.2.1.tgz#3cf37023d199e1c24d1a55b84800c2f3e6468031"
   integrity sha512-Hrgsx+orqoygnmhFbKaHE6c296J+HTAQXoxEF6gNupROmmGJRoyzfG3ccAveqCBrwr/2yxQ5BVd/GTl5agOwSg==
 
+rat-vec@^1.1.1:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/rat-vec/-/rat-vec-1.1.1.tgz#0dde2b66b7b34bb1bcd2a23805eac806d87fd17f"
+  integrity sha1-Dd4rZrezS7G80qI4BerIBth/0X8=
+  dependencies:
+    big-rat "^1.0.3"
+
 raw-body@2.4.0:
   version "2.4.0"
   resolved "https://registry.yarnpkg.com/raw-body/-/raw-body-2.4.0.tgz#a1ce6fb9c9bc356ca52e89256ab59059e13d0332"
@@ -10792,6 +13083,13 @@
   resolved "https://registry.yarnpkg.com/react-error-overlay/-/react-error-overlay-6.0.7.tgz#1dcfb459ab671d53f660a991513cb2f0a0553108"
   integrity sha512-TAv1KJFh3RhqxNvhzxj6LeT5NWklP6rDr2a0jaTfsZ5wSZWHOGeqQyejUp3xxLfPt2UpyJEcVQB/zyPcmonNFA==
 
+react-input-autosize@^2.2.2:
+  version "2.2.2"
+  resolved "https://registry.yarnpkg.com/react-input-autosize/-/react-input-autosize-2.2.2.tgz#fcaa7020568ec206bc04be36f4eb68e647c4d8c2"
+  integrity sha512-jQJgYCA3S0j+cuOwzuCd1OjmBmnZLdqQdiLKRYrsMMzbjUrVDS5RvJUDwJqA7sKuksDuzFtm6hZGKFu7Mjk5aw==
+  dependencies:
+    prop-types "^15.5.8"
+
 react-is@^16.6.0, react-is@^16.7.0, react-is@^16.8.1, react-is@^16.8.4:
   version "16.12.0"
   resolved "https://registry.yarnpkg.com/react-is/-/react-is-16.12.0.tgz#2cc0fe0fba742d97fd527c42a13bec4eeb06241c"
@@ -10812,6 +13110,13 @@
   resolved "https://registry.yarnpkg.com/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz#4f1a273afdfc8f3488a8c516bfda78f872352362"
   integrity sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA==
 
+react-plotly.js@^2.4.0:
+  version "2.4.0"
+  resolved "https://registry.yarnpkg.com/react-plotly.js/-/react-plotly.js-2.4.0.tgz#7a8fd89ffa126daa36a5855890282960e2e4eaf0"
+  integrity sha512-BCkxMe8yWqu3nP/hw9A1KCIuoL67WV5/k68SL9yhEkF6UG+pAuIev9Q3cMKtNkQJZhsYFpOmlqrpPjIdUFACOQ==
+  dependencies:
+    prop-types "^15.7.2"
+
 react-router-dom@^5.0.0:
   version "5.1.2"
   resolved "https://registry.yarnpkg.com/react-router-dom/-/react-router-dom-5.1.2.tgz#06701b834352f44d37fbb6311f870f84c76b9c18"
@@ -10901,6 +13206,20 @@
   optionalDependencies:
     fsevents "2.1.2"
 
+react-select@^3.0.3:
+  version "3.1.0"
+  resolved "https://registry.yarnpkg.com/react-select/-/react-select-3.1.0.tgz#ab098720b2e9fe275047c993f0d0caf5ded17c27"
+  integrity sha512-wBFVblBH1iuCBprtpyGtd1dGMadsG36W5/t2Aj8OE6WbByDg5jIFyT7X5gT+l0qmT5TqWhxX+VsKJvCEl2uL9g==
+  dependencies:
+    "@babel/runtime" "^7.4.4"
+    "@emotion/cache" "^10.0.9"
+    "@emotion/core" "^10.0.9"
+    "@emotion/css" "^10.0.9"
+    memoize-one "^5.0.0"
+    prop-types "^15.6.0"
+    react-input-autosize "^2.2.2"
+    react-transition-group "^4.3.0"
+
 react-slick@~0.25.2:
   version "0.25.2"
   resolved "https://registry.yarnpkg.com/react-slick/-/react-slick-0.25.2.tgz#56331b67d47d8bcfe2dceb6acab1c8fd5bd1f6bc"
@@ -10912,6 +13231,16 @@
     lodash.debounce "^4.0.8"
     resize-observer-polyfill "^1.5.0"
 
+react-transition-group@^4.3.0:
+  version "4.4.1"
+  resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-4.4.1.tgz#63868f9325a38ea5ee9535d828327f85773345c9"
+  integrity sha512-Djqr7OQ2aPUiYurhPalTrVy9ddmFCCzwhqQmtN+J3+3DzLO209Fdr70QrN8Z3DsglWql6iY1lDWAfpFiBtuKGw==
+  dependencies:
+    "@babel/runtime" "^7.5.5"
+    dom-helpers "^5.0.1"
+    loose-envify "^1.4.0"
+    prop-types "^15.6.2"
+
 react@^16.8.6:
   version "16.12.0"
   resolved "https://registry.yarnpkg.com/react/-/react-16.12.0.tgz#0c0a9c6a142429e3614834d5a778e18aa78a0b83"
@@ -10982,7 +13311,7 @@
     parse-json "^5.0.0"
     type-fest "^0.6.0"
 
-"readable-stream@1 || 2", readable-stream@^2.0.0, readable-stream@^2.0.1, readable-stream@^2.0.2, readable-stream@^2.1.5, readable-stream@^2.2.2, readable-stream@^2.3.3, readable-stream@^2.3.6, readable-stream@~2.3.6:
+"readable-stream@1 || 2", readable-stream@^2.0.0, readable-stream@^2.0.1, readable-stream@^2.0.2, readable-stream@^2.1.5, readable-stream@^2.2.2, readable-stream@^2.3.3, readable-stream@^2.3.5, readable-stream@^2.3.6, readable-stream@~2.3.6:
   version "2.3.7"
   resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-2.3.7.tgz#1eca1cf711aef814c04f62252a36a62f6cb23b57"
   integrity sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==
@@ -10995,6 +13324,25 @@
     string_decoder "~1.1.1"
     util-deprecate "~1.0.1"
 
+"readable-stream@>=1.0.33-1 <1.1.0-0", readable-stream@~1.0.17, readable-stream@~1.0.27-1:
+  version "1.0.34"
+  resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-1.0.34.tgz#125820e34bc842d2f2aaafafe4c2916ee32c157c"
+  integrity sha1-Elgg40vIQtLyqq+v5MKRbuMsFXw=
+  dependencies:
+    core-util-is "~1.0.0"
+    inherits "~2.0.1"
+    isarray "0.0.1"
+    string_decoder "~0.10.x"
+
+readable-stream@^3.0.2:
+  version "3.6.0"
+  resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-3.6.0.tgz#337bbda3adc0706bd3e024426a286d4b4b2c9198"
+  integrity sha512-BViHy7LKeTz4oNnkcLJ+lVSL6vpiFeX6/d3oSH8zCW7UxP2onchk+vTGB143xuFjHS3deTgkKoXXymXqymiIdA==
+  dependencies:
+    inherits "^2.0.3"
+    string_decoder "^1.1.1"
+    util-deprecate "^1.0.1"
+
 readable-stream@^3.0.6, readable-stream@^3.1.1:
   version "3.4.0"
   resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-3.4.0.tgz#a51c26754658e0a3c21dbf59163bd45ba6f447fc"
@@ -11004,6 +13352,16 @@
     string_decoder "^1.1.1"
     util-deprecate "^1.0.1"
 
+readable-stream@~1.1.9:
+  version "1.1.14"
+  resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-1.1.14.tgz#7cf4c54ef648e3813084c636dd2079e166c081d9"
+  integrity sha1-fPTFTvZI44EwhMY23SB54WbAgdk=
+  dependencies:
+    core-util-is "~1.0.0"
+    inherits "~2.0.1"
+    isarray "0.0.1"
+    string_decoder "~0.10.x"
+
 readdirp@^2.2.1:
   version "2.2.1"
   resolved "https://registry.yarnpkg.com/readdirp/-/readdirp-2.2.1.tgz#0e87622a3325aa33e892285caf8b4e846529a525"
@@ -11042,6 +13400,15 @@
     indent-string "^3.0.0"
     strip-indent "^2.0.0"
 
+reduce-simplicial-complex@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/reduce-simplicial-complex/-/reduce-simplicial-complex-1.0.0.tgz#74d696a2f835f7a6dcd92065fd8c5181f2edf8bc"
+  integrity sha1-dNaWovg196bc2SBl/YxRgfLt+Lw=
+  dependencies:
+    cell-orientation "^1.0.1"
+    compare-cell "^1.0.0"
+    compare-oriented-cell "^1.0.1"
+
 regenerate-unicode-properties@^8.2.0:
   version "8.2.0"
   resolved "https://registry.yarnpkg.com/regenerate-unicode-properties/-/regenerate-unicode-properties-8.2.0.tgz#e5de7111d655e7ba60c057dbe9ff37c87e65cdec"
@@ -11090,6 +13457,11 @@
   resolved "https://registry.yarnpkg.com/regex-parser/-/regex-parser-2.2.10.tgz#9e66a8f73d89a107616e63b39d4deddfee912b37"
   integrity sha512-8t6074A68gHfU8Neftl0Le6KTDwfGAj7IyjPIMSfikI2wJUTHDMaIq42bUsfVnj8mhx0R+45rdUXHGpN164avA==
 
+regex-regex@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/regex-regex/-/regex-regex-1.0.0.tgz#9048a1eaeb870f4d480dabc76fc42cdcc0bc3a72"
+  integrity sha1-kEih6uuHD01IDavHb8Qs3MC8OnI=
+
 regexp-tree@^0.1.21, regexp-tree@~0.1.1:
   version "0.1.21"
   resolved "https://registry.yarnpkg.com/regexp-tree/-/regexp-tree-0.1.21.tgz#55e2246b7f7d36f1b461490942fa780299c400d7"
@@ -11113,7 +13485,7 @@
   resolved "https://registry.yarnpkg.com/regexpp/-/regexpp-3.1.0.tgz#206d0ad0a5648cffbdb8ae46438f3dc51c9f78e2"
   integrity sha512-ZOIzd8yVsQQA7j8GCSlPGXwg5PfmA1mrq0JP4nGhh54LaKN3xdai/vHUDu74pKwV8OxseMS65u2NImosQcSD0Q==
 
-regexpu-core@^4.7.0:
+regexpu-core@^4.5.4, regexpu-core@^4.7.0:
   version "4.7.0"
   resolved "https://registry.yarnpkg.com/regexpu-core/-/regexpu-core-4.7.0.tgz#fcbf458c50431b0bb7b45d6967b8192d91f3d938"
   integrity sha512-TQ4KXRnIn6tz6tjnrXEkD/sshygKH/j5KzK86X8MkeHyZ8qst/LZ89j3X4/8HEIfHANTFIP/AbXakeRhWIl5YQ==
@@ -11152,6 +13524,88 @@
   dependencies:
     jsesc "~0.5.0"
 
+regl-error2d@^2.0.8:
+  version "2.0.8"
+  resolved "https://registry.yarnpkg.com/regl-error2d/-/regl-error2d-2.0.8.tgz#0f26371ee99c78d42e9c5a197387e32034bcf640"
+  integrity sha512-5nszdicXbimRUnYB42i+O7KPcla7PzI62nZLCP6qVRKlQCf3rSrWbikMNd1S84LE8+deWHWcb8rZ/v7rZ9qmmw==
+  dependencies:
+    array-bounds "^1.0.1"
+    bubleify "^1.2.0"
+    color-normalize "^1.5.0"
+    flatten-vertex-data "^1.0.2"
+    object-assign "^4.1.1"
+    pick-by-alias "^1.2.0"
+    to-float32 "^1.0.1"
+    update-diff "^1.1.0"
+
+regl-line2d@^3.0.15:
+  version "3.0.15"
+  resolved "https://registry.yarnpkg.com/regl-line2d/-/regl-line2d-3.0.15.tgz#5baf2a0ed5024cec2ec038c6ada601c98ac579e3"
+  integrity sha512-RuQbg9iZ6MyuInG8izF6zjQ/2g4qL6sg1egiuFalWzaGSvuve/IWBsIcqKTlwpiEsRt9b4cHu9NYs2fLt1gYJw==
+  dependencies:
+    array-bounds "^1.0.1"
+    array-normalize "^1.1.4"
+    bubleify "^1.2.0"
+    color-normalize "^1.5.0"
+    earcut "^2.1.5"
+    es6-weak-map "^2.0.3"
+    flatten-vertex-data "^1.0.2"
+    glslify "^7.0.0"
+    object-assign "^4.1.1"
+    parse-rect "^1.2.0"
+    pick-by-alias "^1.2.0"
+    to-float32 "^1.0.1"
+
+regl-scatter2d@^3.1.2, regl-scatter2d@^3.1.8:
+  version "3.1.8"
+  resolved "https://registry.yarnpkg.com/regl-scatter2d/-/regl-scatter2d-3.1.8.tgz#5fdf77f9cee7e71497d038dc1b654dc4340b6bfb"
+  integrity sha512-Z9MYAUx9t8e3MsiHBbJAEstbIqauXxzcL9DmuKXQuRWfCMF2DBytYJtE0FpbQU6639wEMAJ54SEIlISWF8sQ2g==
+  dependencies:
+    array-range "^1.0.1"
+    array-rearrange "^2.2.2"
+    clamp "^1.0.1"
+    color-id "^1.1.0"
+    color-normalize "1.5.0"
+    color-rgba "^2.1.1"
+    flatten-vertex-data "^1.0.2"
+    glslify "^7.0.0"
+    image-palette "^2.1.0"
+    is-iexplorer "^1.0.0"
+    object-assign "^4.1.1"
+    parse-rect "^1.2.0"
+    pick-by-alias "^1.2.0"
+    point-cluster "^3.1.8"
+    to-float32 "^1.0.1"
+    update-diff "^1.1.0"
+
+regl-splom@^1.0.8:
+  version "1.0.8"
+  resolved "https://registry.yarnpkg.com/regl-splom/-/regl-splom-1.0.8.tgz#05f165a6f0b8afc6b6b97fafa775282d44387db3"
+  integrity sha512-4GQTgcArCbGLsXhgalWVBxeW7OXllnu+Gvil/4SbQQmtiqLCl+xgF79pISKY9mLXTlobxiX7cVKdjGjp25559A==
+  dependencies:
+    array-bounds "^1.0.1"
+    array-range "^1.0.1"
+    bubleify "^1.2.0"
+    color-alpha "^1.0.4"
+    defined "^1.0.0"
+    flatten-vertex-data "^1.0.2"
+    left-pad "^1.3.0"
+    parse-rect "^1.2.0"
+    pick-by-alias "^1.2.0"
+    point-cluster "^3.1.8"
+    raf "^3.4.1"
+    regl-scatter2d "^3.1.2"
+
+regl@1.3.11:
+  version "1.3.11"
+  resolved "https://registry.yarnpkg.com/regl/-/regl-1.3.11.tgz#515e5173ffdc0618f908dd4e338a34ae9555f745"
+  integrity sha512-tmt6CRhRqbcsYDWNwv+iG7GGOXdgoOBC7lKzoPMgnzpt3WKBQ3c8i7AxgbvTRZzty29hrW92fAJeZkPFQehfWA==
+
+regl@^1.3.11:
+  version "1.6.1"
+  resolved "https://registry.yarnpkg.com/regl/-/regl-1.6.1.tgz#6930172cda9b8fb65724abc0d4930d79333f5460"
+  integrity sha512-7Z9rmpEqmLNwC9kCYCyfyu47eWZaQWeNpwZfwz99QueXN8B/Ow40DB0N+OeUeM/yu9pZAB01+JgJ+XghGveVoA==
+
 relateurl@^0.2.7:
   version "0.2.7"
   resolved "https://registry.yarnpkg.com/relateurl/-/relateurl-0.2.7.tgz#54dbf377e51440aca90a4cd274600d3ff2d888a9"
@@ -11178,7 +13632,7 @@
   resolved "https://registry.yarnpkg.com/repeat-element/-/repeat-element-1.1.3.tgz#782e0d825c0c5a3bb39731f84efee6b742e6b1ce"
   integrity sha512-ahGq0ZnV5m5XtZLMb+vP76kcAM5nkLqk0lpqAuojSKGgQtn4eRi4ZZGm2olo2zKFH+sMsWaqOCW1dqAnOru72g==
 
-repeat-string@^1.6.1:
+repeat-string@^1.3.0, repeat-string@^1.5.2, repeat-string@^1.6.1:
   version "1.6.1"
   resolved "https://registry.yarnpkg.com/repeat-string/-/repeat-string-1.6.1.tgz#8dcae470e1c88abc2d600fff4a776286da75e637"
   integrity sha1-jcrkcOHIirwtYA//Sndihtp15jc=
@@ -11289,6 +13743,13 @@
   resolved "https://registry.yarnpkg.com/resolve-pathname/-/resolve-pathname-3.0.0.tgz#99d02224d3cf263689becbb393bc560313025dcd"
   integrity sha512-C7rARubxI8bXFNB/hqcp/4iUeIXJhJZvFPFPiSPRnhU5UPxzMFIl+2E6yY6c4k9giDJAhtV+enfA+G89N6Csng==
 
+resolve-protobuf-schema@^2.1.0:
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/resolve-protobuf-schema/-/resolve-protobuf-schema-2.1.0.tgz#9ca9a9e69cf192bbdaf1006ec1973948aa4a3758"
+  integrity sha512-kI5ffTiZWmJaS/huM8wZfEMer1eRd7oJQhDuxeCLe3t7N7mX3z94CN0xPxBQxFYQTSNz9T0i+v6inKqSdK8xrQ==
+  dependencies:
+    protocol-buffers-schema "^3.3.1"
+
 resolve-url-loader@3.1.1:
   version "3.1.1"
   resolved "https://registry.yarnpkg.com/resolve-url-loader/-/resolve-url-loader-3.1.1.tgz#28931895fa1eab9be0647d3b2958c100ae3c0bf0"
@@ -11322,6 +13783,18 @@
   dependencies:
     path-parse "^1.0.6"
 
+resolve@^0.6.1:
+  version "0.6.3"
+  resolved "https://registry.yarnpkg.com/resolve/-/resolve-0.6.3.tgz#dd957982e7e736debdf53b58a4dd91754575dd46"
+  integrity sha1-3ZV5gufnNt699TtYpN2RdUV13UY=
+
+resolve@^1.0.0, resolve@^1.1.5, resolve@^1.10.1, resolve@^1.12.0, resolve@^1.13.1, resolve@^1.15.1, resolve@~1.17.0:
+  version "1.17.0"
+  resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.17.0.tgz#b25941b54968231cc2d1bb76a79cb7f2c0bf8444"
+  integrity sha512-ic+7JYiV8Vi2yzQGFWOkiZD5Z9z7O2Zhm9XMaTxdJExKasieFCr+yXZ/WmXsckHiKl12ar0y6XiXDx3m4RHn1w==
+  dependencies:
+    path-parse "^1.0.6"
+
 resolve@^1.10.0, resolve@^1.3.2, resolve@^1.5.0, resolve@^1.8.1:
   version "1.14.2"
   resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.14.2.tgz#dbf31d0fa98b1f29aa5169783b9c290cb865fea2"
@@ -11329,13 +13802,6 @@
   dependencies:
     path-parse "^1.0.6"
 
-resolve@^1.10.1, resolve@^1.12.0, resolve@^1.13.1, resolve@^1.15.1:
-  version "1.17.0"
-  resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.17.0.tgz#b25941b54968231cc2d1bb76a79cb7f2c0bf8444"
-  integrity sha512-ic+7JYiV8Vi2yzQGFWOkiZD5Z9z7O2Zhm9XMaTxdJExKasieFCr+yXZ/WmXsckHiKl12ar0y6XiXDx3m4RHn1w==
-  dependencies:
-    path-parse "^1.0.6"
-
 responselike@^1.0.2:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/responselike/-/responselike-1.0.2.tgz#918720ef3b631c5642be068f15ade5a46f4ba1e7"
@@ -11351,6 +13817,13 @@
     onetime "^5.1.0"
     signal-exit "^3.0.2"
 
+resumer@~0.0.0:
+  version "0.0.0"
+  resolved "https://registry.yarnpkg.com/resumer/-/resumer-0.0.0.tgz#f1e8f461e4064ba39e82af3cdc2a8c893d076759"
+  integrity sha1-8ej0YeQGS6Oegq883CqMiT0HZ1k=
+  dependencies:
+    through "~2.3.4"
+
 ret@~0.1.10:
   version "0.1.15"
   resolved "https://registry.yarnpkg.com/ret/-/ret-0.1.15.tgz#b8a4825d5bdb1fc3f6f53c2bc33f81388681c7bc"
@@ -11384,6 +13857,18 @@
   resolved "https://registry.yarnpkg.com/rgba-regex/-/rgba-regex-1.0.0.tgz#43374e2e2ca0968b0ef1523460b7d730ff22eeb3"
   integrity sha1-QzdOLiyglosO8VI0YLfXMP8i7rM=
 
+right-align@^0.1.1:
+  version "0.1.3"
+  resolved "https://registry.yarnpkg.com/right-align/-/right-align-0.1.3.tgz#61339b722fe6a3515689210d24e14c96148613ef"
+  integrity sha1-YTObci/mo1FWiSENJOFMlhSGE+8=
+  dependencies:
+    align-text "^0.1.1"
+
+right-now@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/right-now/-/right-now-1.0.0.tgz#6e89609deebd7dcdaf8daecc9aea39cf585a0918"
+  integrity sha1-bolgne69fc2vja7Mmuo5z1haCRg=
+
 rimraf@2.6.3:
   version "2.6.3"
   resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-2.6.3.tgz#b2d104fe0d8fb27cf9e0a1cda8262dd3833c6cab"
@@ -11414,6 +13899,89 @@
     babel-runtime "6.x"
     classnames "^2.2.5"
 
+robust-compress@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-compress/-/robust-compress-1.0.0.tgz#4cf62c4b318d8308516012bb8c11752f39329b1b"
+  integrity sha1-TPYsSzGNgwhRYBK7jBF1Lzkymxs=
+
+robust-determinant@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/robust-determinant/-/robust-determinant-1.1.0.tgz#8ecae79b79caab3e74f6debe2237e5391a27e9c7"
+  integrity sha1-jsrnm3nKqz509t6+IjflORon6cc=
+  dependencies:
+    robust-compress "^1.0.0"
+    robust-scale "^1.0.0"
+    robust-sum "^1.0.0"
+    two-product "^1.0.0"
+
+robust-dot-product@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-dot-product/-/robust-dot-product-1.0.0.tgz#c9ba0178bd2c304bfd725f58e889f1d946004553"
+  integrity sha1-yboBeL0sMEv9cl9Y6Inx2UYARVM=
+  dependencies:
+    robust-sum "^1.0.0"
+    two-product "^1.0.0"
+
+robust-in-sphere@^1.1.3:
+  version "1.1.3"
+  resolved "https://registry.yarnpkg.com/robust-in-sphere/-/robust-in-sphere-1.1.3.tgz#1c5883d16a4e923929476ef34819857bf2a9cf75"
+  integrity sha1-HFiD0WpOkjkpR27zSBmFe/Kpz3U=
+  dependencies:
+    robust-scale "^1.0.0"
+    robust-subtract "^1.0.0"
+    robust-sum "^1.0.0"
+    two-product "^1.0.0"
+
+robust-linear-solve@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-linear-solve/-/robust-linear-solve-1.0.0.tgz#0cd6ac5040691a6f2aa3cd6311d728905ca3a1f1"
+  integrity sha1-DNasUEBpGm8qo81jEdcokFyjofE=
+  dependencies:
+    robust-determinant "^1.1.0"
+
+robust-orientation@^1.0.1, robust-orientation@^1.0.2, robust-orientation@^1.1.2, robust-orientation@^1.1.3:
+  version "1.1.3"
+  resolved "https://registry.yarnpkg.com/robust-orientation/-/robust-orientation-1.1.3.tgz#daff5b00d3be4e60722f0e9c0156ef967f1c2049"
+  integrity sha1-2v9bANO+TmByLw6cAVbvln8cIEk=
+  dependencies:
+    robust-scale "^1.0.2"
+    robust-subtract "^1.0.0"
+    robust-sum "^1.0.0"
+    two-product "^1.0.2"
+
+robust-product@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-product/-/robust-product-1.0.0.tgz#685250007cdbba7cf1de75bff6d2927011098abe"
+  integrity sha1-aFJQAHzbunzx3nW/9tKScBEJir4=
+  dependencies:
+    robust-scale "^1.0.0"
+    robust-sum "^1.0.0"
+
+robust-scale@^1.0.0, robust-scale@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/robust-scale/-/robust-scale-1.0.2.tgz#775132ed09542d028e58b2cc79c06290bcf78c32"
+  integrity sha1-d1Ey7QlULQKOWLLMecBikLz3jDI=
+  dependencies:
+    two-product "^1.0.2"
+    two-sum "^1.0.0"
+
+robust-segment-intersect@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/robust-segment-intersect/-/robust-segment-intersect-1.0.1.tgz#3252b6a0fc1ba14ade6915ccbe09cbce9aab1c1c"
+  integrity sha1-MlK2oPwboUreaRXMvgnLzpqrHBw=
+  dependencies:
+    robust-orientation "^1.1.3"
+
+robust-subtract@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-subtract/-/robust-subtract-1.0.0.tgz#e0b164e1ed8ba4e3a5dda45a12038348dbed3e9a"
+  integrity sha1-4LFk4e2LpOOl3aRaEgODSNvtPpo=
+
+robust-sum@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/robust-sum/-/robust-sum-1.0.0.tgz#16646e525292b4d25d82757a286955e0bbfa53d9"
+  integrity sha1-FmRuUlKStNJdgnV6KGlV4Lv6U9k=
+
 rsvp@^4.8.4:
   version "4.8.5"
   resolved "https://registry.yarnpkg.com/rsvp/-/rsvp-4.8.5.tgz#c8f155311d167f68f21e168df71ec5b083113734"
@@ -11438,6 +14006,11 @@
   dependencies:
     aproba "^1.1.1"
 
+rw@^1.3.3:
+  version "1.3.3"
+  resolved "https://registry.yarnpkg.com/rw/-/rw-1.3.3.tgz#3f862dfa91ab766b14885ef4d01124bfda074fb4"
+  integrity sha1-P4Yt+pGrdmsUiF700BEkv9oHT7Q=
+
 rxjs@^6.5.3:
   version "6.5.5"
   resolved "https://registry.yarnpkg.com/rxjs/-/rxjs-6.5.5.tgz#c5c884e3094c8cfee31bf27eb87e54ccfc87f9ec"
@@ -11474,6 +14047,11 @@
   resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a"
   integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==
 
+sane-topojson@^4.0.0:
+  version "4.0.0"
+  resolved "https://registry.yarnpkg.com/sane-topojson/-/sane-topojson-4.0.0.tgz#624cdb26fc6d9392c806897bfd1a393f29bb5308"
+  integrity sha512-bJILrpBboQfabG3BNnHI2hZl52pbt80BE09u4WhnrmzuF2JbMKZdl62G5glXskJ46p+gxE2IzOwGj/awR4g8AA==
+
 sane@^4.0.3:
   version "4.1.0"
   resolved "https://registry.yarnpkg.com/sane/-/sane-4.1.0.tgz#ed881fd922733a6c461bc189dc2b6c006f3ffded"
@@ -11700,6 +14278,11 @@
   dependencies:
     kind-of "^6.0.2"
 
+shallow-copy@0.0.1, shallow-copy@~0.0.1:
+  version "0.0.1"
+  resolved "https://registry.yarnpkg.com/shallow-copy/-/shallow-copy-0.0.1.tgz#415f42702d73d810330292cc5ee86eae1a11a170"
+  integrity sha1-QV9CcC1z2BAzApLMXuhurhoRoXA=
+
 shallow-equal@^1.0.0:
   version "1.2.1"
   resolved "https://registry.yarnpkg.com/shallow-equal/-/shallow-equal-1.2.1.tgz#4c16abfa56043aa20d050324efa68940b0da79da"
@@ -11757,6 +14340,16 @@
   resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.2.tgz#b5fdc08f1287ea1178628e415e25132b73646c6d"
   integrity sha1-tf3AjxKH6hF4Yo5BXiUTK3NkbG0=
 
+signum@^0.0.0:
+  version "0.0.0"
+  resolved "https://registry.yarnpkg.com/signum/-/signum-0.0.0.tgz#ab551b1003351070a704783f1a09c5e7691f9cf6"
+  integrity sha1-q1UbEAM1EHCnBHg/GgnF52kfnPY=
+
+signum@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/signum/-/signum-1.0.0.tgz#74a7d2bf2a20b40eba16a92b152124f1d559fa77"
+  integrity sha1-dKfSvyogtA66FqkrFSEk8dVZ+nc=
+
 simple-swizzle@^0.2.2:
   version "0.2.2"
   resolved "https://registry.yarnpkg.com/simple-swizzle/-/simple-swizzle-0.2.2.tgz#a4da6b635ffcccca33f70d17cb92592de95e557a"
@@ -11764,11 +14357,62 @@
   dependencies:
     is-arrayish "^0.3.1"
 
+simplicial-complex-boundary@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/simplicial-complex-boundary/-/simplicial-complex-boundary-1.0.1.tgz#72c9ff1e24deaa374c9bb2fa0cbf0c081ebef815"
+  integrity sha1-csn/HiTeqjdMm7L6DL8MCB6++BU=
+  dependencies:
+    boundary-cells "^2.0.0"
+    reduce-simplicial-complex "^1.0.0"
+
+simplicial-complex-contour@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/simplicial-complex-contour/-/simplicial-complex-contour-1.0.2.tgz#890aacac284365340110545cf2629a26e04bf9d1"
+  integrity sha1-iQqsrChDZTQBEFRc8mKaJuBL+dE=
+  dependencies:
+    marching-simplex-table "^1.0.0"
+    ndarray "^1.0.15"
+    ndarray-sort "^1.0.0"
+    typedarray-pool "^1.1.0"
+
+simplicial-complex@^0.3.3:
+  version "0.3.3"
+  resolved "https://registry.yarnpkg.com/simplicial-complex/-/simplicial-complex-0.3.3.tgz#4c30cad57f9e45729dd8f306c8753579f46be99e"
+  integrity sha1-TDDK1X+eRXKd2PMGyHU1efRr6Z4=
+  dependencies:
+    bit-twiddle "~0.0.1"
+    union-find "~0.0.3"
+
+simplicial-complex@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/simplicial-complex/-/simplicial-complex-1.0.0.tgz#6c33a4ed69fcd4d91b7bcadd3b30b63683eae241"
+  integrity sha1-bDOk7Wn81Nkbe8rdOzC2NoPq4kE=
+  dependencies:
+    bit-twiddle "^1.0.0"
+    union-find "^1.0.0"
+
+simplify-planar-graph@^2.0.1:
+  version "2.0.1"
+  resolved "https://registry.yarnpkg.com/simplify-planar-graph/-/simplify-planar-graph-2.0.1.tgz#bc85893725f32e8fa8ae25681398446d2cbcf766"
+  integrity sha1-vIWJNyXzLo+oriVoE5hEbSy892Y=
+  dependencies:
+    robust-orientation "^1.0.1"
+    simplicial-complex "^0.3.3"
+
 sisteransi@^1.0.3:
   version "1.0.4"
   resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-1.0.4.tgz#386713f1ef688c7c0304dc4c0632898941cad2e3"
   integrity sha512-/ekMoM4NJ59ivGSfKapeG+FWtrmWvA1p6FBZwXrqojw90vJu8lBmrTxCMuBCydKtkaUe2zt4PlxeTKpjwMbyig==
 
+slab-decomposition@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/slab-decomposition/-/slab-decomposition-1.0.2.tgz#1ded56754d408b10739f145103dfc61807f65134"
+  integrity sha1-He1WdU1AixBznxRRA9/GGAf2UTQ=
+  dependencies:
+    binary-search-bounds "^1.0.0"
+    functional-red-black-tree "^1.0.0"
+    robust-orientation "^1.1.3"
+
 slash@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/slash/-/slash-1.0.0.tgz#c41f2f6c39fc16d1cd17ad4b5d896114ae470d55"
@@ -11884,11 +14528,28 @@
   resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263"
   integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==
 
-source-map@^0.5.0, source-map@^0.5.6:
+"source-map@>= 0.1.2":
+  version "0.7.3"
+  resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.7.3.tgz#5302f8169031735226544092e64981f751750383"
+  integrity sha512-CkCj6giN3S+n9qrYiBTX5gystlENnRW5jZeNLHpe6aue+SrHcG5VYwujhW9s4dY31mEGsxBDrHR6oI69fTXsaQ==
+
+source-map@^0.5.0, source-map@^0.5.6, source-map@^0.5.7, source-map@~0.5.1:
   version "0.5.7"
   resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.5.7.tgz#8a039d2d1021d22d1ea14c80d8ea468ba2ef3fcc"
   integrity sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=
 
+source-map@~0.1.33:
+  version "0.1.43"
+  resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.1.43.tgz#c24bc146ca517c1471f5dacbe2571b2b7f9e3346"
+  integrity sha1-wkvBRspRfBRx9drL4lcbK3+eM0Y=
+  dependencies:
+    amdefine ">=0.0.4"
+
+sourcemap-codec@^1.4.4:
+  version "1.4.8"
+  resolved "https://registry.yarnpkg.com/sourcemap-codec/-/sourcemap-codec-1.4.8.tgz#ea804bd94857402e6992d05a38ef1ae35a9ab4c4"
+  integrity sha512-9NykojV5Uih4lgo5So5dtw+f0JgJX30KCNI8gwhz2J9A15wD0Ml6tjHKwf6fTSa6fAdVBdZeNOs9eJ71qCk8vA==
+
 spdx-correct@^3.0.0:
   version "3.1.0"
   resolved "https://registry.yarnpkg.com/spdx-correct/-/spdx-correct-3.1.0.tgz#fb83e504445268f154b074e218c87c003cd31df4"
@@ -11938,6 +14599,14 @@
     select-hose "^2.0.0"
     spdy-transport "^3.0.0"
 
+split-polygon@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/split-polygon/-/split-polygon-1.0.0.tgz#0eacc8a136a76b12a3d95256ea7da45db0c2d247"
+  integrity sha1-DqzIoTanaxKj2VJW6n2kXbDC0kc=
+  dependencies:
+    robust-dot-product "^1.0.0"
+    robust-sum "^1.0.0"
+
 split-string@^3.0.1, split-string@^3.0.2:
   version "3.1.0"
   resolved "https://registry.yarnpkg.com/split-string/-/split-string-3.1.0.tgz#7cb09dda3a86585705c64b39a6466038682e8fe2"
@@ -11945,6 +14614,11 @@
   dependencies:
     extend-shallow "^3.0.0"
 
+sprintf-js@^1.0.3:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.1.2.tgz#da1765262bf8c0f571749f2ad6c26300207ae673"
+  integrity sha512-VE0SOVEHCk7Qc8ulkWw3ntAzXuqf7S2lvwQaDLRnUeIEaKNQJzV6BwmLKhOqT61aGhfUMrXeaBk+oDGCzvhcug==
+
 sprintf-js@~1.0.2:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c"
@@ -11985,11 +14659,30 @@
   resolved "https://registry.yarnpkg.com/stable/-/stable-0.1.8.tgz#836eb3c8382fe2936feaf544631017ce7d47a3cf"
   integrity sha512-ji9qxRnOVfcuLDySj9qzhGSEFVobyt1kIOSkj1qZzYLzq7Tos/oUUWvotUPQLlrsidqsK6tBH89Bc9kL5zHA6w==
 
+stack-trace@0.0.9:
+  version "0.0.9"
+  resolved "https://registry.yarnpkg.com/stack-trace/-/stack-trace-0.0.9.tgz#a8f6eaeca90674c333e7c43953f275b451510695"
+  integrity sha1-qPbq7KkGdMMz58Q5U/J1tFFRBpU=
+
 stack-utils@^1.0.1:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/stack-utils/-/stack-utils-1.0.2.tgz#33eba3897788558bebfc2db059dc158ec36cebb8"
   integrity sha512-MTX+MeG5U994cazkjd/9KNAapsHnibjMLnfXodlkXw76JEea0UiNzrqidzo1emMwk7w5Qhc9jd4Bn9TBb1MFwA==
 
+static-eval@^2.0.0:
+  version "2.0.5"
+  resolved "https://registry.yarnpkg.com/static-eval/-/static-eval-2.0.5.tgz#f0782e66999c4b3651cda99d9ce59c507d188f71"
+  integrity sha512-nNbV6LbGtMBgv7e9LFkt5JV8RVlRsyJrphfAt9tOtBBW/SfnzZDf2KnS72an8e434A+9e/BmJuTxeGPvrAK7KA==
+  dependencies:
+    escodegen "^1.11.1"
+
+static-eval@~0.2.0:
+  version "0.2.4"
+  resolved "https://registry.yarnpkg.com/static-eval/-/static-eval-0.2.4.tgz#b7d34d838937b969f9641ca07d48f8ede263ea7b"
+  integrity sha1-t9NNg4k3uWn5ZBygfUj47eJj6ns=
+  dependencies:
+    escodegen "~0.0.24"
+
 static-extend@^0.1.1:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/static-extend/-/static-extend-0.1.2.tgz#60809c39cbff55337226fd5e0b520f341f1fb5c6"
@@ -11998,6 +14691,23 @@
     define-property "^0.2.5"
     object-copy "^0.1.0"
 
+static-module@^1.0.0:
+  version "1.5.0"
+  resolved "https://registry.yarnpkg.com/static-module/-/static-module-1.5.0.tgz#27da9883c41a8cd09236f842f0c1ebc6edf63d86"
+  integrity sha1-J9qYg8QajNCSNvhC8MHrxu32PYY=
+  dependencies:
+    concat-stream "~1.6.0"
+    duplexer2 "~0.0.2"
+    escodegen "~1.3.2"
+    falafel "^2.1.0"
+    has "^1.0.0"
+    object-inspect "~0.4.0"
+    quote-stream "~0.0.0"
+    readable-stream "~1.0.27-1"
+    shallow-copy "~0.0.1"
+    static-eval "~0.2.0"
+    through2 "~0.4.1"
+
 "statuses@>= 1.4.0 < 2", "statuses@>= 1.5.0 < 2", statuses@~1.5.0:
   version "1.5.0"
   resolved "https://registry.yarnpkg.com/statuses/-/statuses-1.5.0.tgz#161c7dac177659fd9811f43771fa99381478628c"
@@ -12073,6 +14783,21 @@
     astral-regex "^1.0.0"
     strip-ansi "^5.2.0"
 
+string-split-by@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/string-split-by/-/string-split-by-1.0.0.tgz#53895fb3397ebc60adab1f1e3a131f5372586812"
+  integrity sha512-KaJKY+hfpzNyet/emP81PJA9hTVSfxNLS9SFTWxdCnnW1/zOOwiV248+EfoX7IQFcBaOp4G5YE6xTJMF+pLg6A==
+  dependencies:
+    parenthesis "^3.1.5"
+
+string-to-arraybuffer@^1.0.0:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/string-to-arraybuffer/-/string-to-arraybuffer-1.0.2.tgz#161147fbadea02e28b0935002cec4c40f1ca7f0a"
+  integrity sha512-DaGZidzi93dwjQen5I2osxR9ERS/R7B1PFyufNMnzhj+fmlDQAc1DSDIJVJhgI8Oq221efIMbABUBdPHDRt43Q==
+  dependencies:
+    atob-lite "^2.0.0"
+    is-base64 "^0.1.0"
+
 string-width@^1.0.1:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/string-width/-/string-width-1.0.2.tgz#118bdf5b8cdc51a2a7e70d211e07e2b0b9b107d3"
@@ -12128,6 +14853,15 @@
     define-properties "^1.1.3"
     es-abstract "^1.17.0-next.1"
 
+string.prototype.trim@~1.2.1:
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/string.prototype.trim/-/string.prototype.trim-1.2.1.tgz#141233dff32c82bfad80684d7e5f0869ee0fb782"
+  integrity sha512-MjGFEeqixw47dAMFMtgUro/I0+wNqZB5GKXGt1fFr24u3TzDXCPu7J9Buppzoe3r/LqkSDLDDJzE15RGWDGAVw==
+  dependencies:
+    define-properties "^1.1.3"
+    es-abstract "^1.17.0-next.1"
+    function-bind "^1.1.1"
+
 string.prototype.trimleft@^2.1.1:
   version "2.1.1"
   resolved "https://registry.yarnpkg.com/string.prototype.trimleft/-/string.prototype.trimleft-2.1.1.tgz#9bdb8ac6abd6d602b17a4ed321870d2f8dcefc74"
@@ -12151,6 +14885,11 @@
   dependencies:
     safe-buffer "~5.2.0"
 
+string_decoder@~0.10.x:
+  version "0.10.31"
+  resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-0.10.31.tgz#62e203bc41766c6c28c9fc84301dab1c5310fa94"
+  integrity sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=
+
 string_decoder@~1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.1.1.tgz#9cf1611ba62685d7030ae9e4ba34149c3af03fc8"
@@ -12228,6 +14967,11 @@
   resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-2.0.1.tgz#3c531942e908c2697c0ec344858c286c7ca0a60a"
   integrity sha1-PFMZQukIwml8DsNEhYwobHygpgo=
 
+strongly-connected-components@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/strongly-connected-components/-/strongly-connected-components-1.0.1.tgz#0920e2b4df67c8eaee96c6b6234fe29e873dba99"
+  integrity sha1-CSDitN9nyOrulsa2I0/inoc9upk=
+
 style-loader@0.23.1:
   version "0.23.1"
   resolved "https://registry.yarnpkg.com/style-loader/-/style-loader-0.23.1.tgz#cb9154606f3e771ab6c4ab637026a1049174d925"
@@ -12245,6 +14989,18 @@
     postcss "^7.0.0"
     postcss-selector-parser "^3.0.0"
 
+supercluster@^7.0.0:
+  version "7.1.0"
+  resolved "https://registry.yarnpkg.com/supercluster/-/supercluster-7.1.0.tgz#f0a457426ec0ab95d69c5f03b51e049774b94479"
+  integrity sha512-LDasImUAFMhTqhK+cUXfy9C2KTUqJ3gucLjmNLNFmKWOnDUBxLFLH9oKuXOTCLveecmxh8fbk8kgh6Q0gsfe2w==
+  dependencies:
+    kdbush "^3.0.0"
+
+superscript-text@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/superscript-text/-/superscript-text-1.0.0.tgz#e7cb2752567360df50beb0610ce8df3d71d8dfd8"
+  integrity sha1-58snUlZzYN9QvrBhDOjfPXHY39g=
+
 supports-color@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-2.0.0.tgz#535d045ce6b6363fa40117084629995e9df324c7"
@@ -12279,11 +15035,46 @@
     has-flag "^4.0.0"
     supports-color "^7.0.0"
 
+surface-nets@^1.0.0, surface-nets@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/surface-nets/-/surface-nets-1.0.2.tgz#e433c8cbba94a7274c6f4c99552b461bf1fc7a4b"
+  integrity sha1-5DPIy7qUpydMb0yZVStGG/H8eks=
+  dependencies:
+    ndarray-extract-contour "^1.0.0"
+    triangulate-hypercube "^1.0.0"
+    zero-crossings "^1.0.0"
+
+svg-arc-to-cubic-bezier@^3.0.0:
+  version "3.2.0"
+  resolved "https://registry.yarnpkg.com/svg-arc-to-cubic-bezier/-/svg-arc-to-cubic-bezier-3.2.0.tgz#390c450035ae1c4a0104d90650304c3bc814abe6"
+  integrity sha512-djbJ/vZKZO+gPoSDThGNpKDO+o+bAeA4XQKovvkNCqnIS2t+S4qnLAGQhyyrulhCFRl1WWzAp0wUDV8PpTVU3g==
+
 svg-parser@^2.0.0:
   version "2.0.2"
   resolved "https://registry.yarnpkg.com/svg-parser/-/svg-parser-2.0.2.tgz#d134cc396fa2681dc64f518330784e98bd801ec8"
   integrity sha512-1gtApepKFweigFZj3sGO8KT8LvVZK8io146EzXrpVuWCDAbISz/yMucco3hWTkpZNoPabM+dnMOpy6Swue68Zg==
 
+svg-path-bounds@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/svg-path-bounds/-/svg-path-bounds-1.0.1.tgz#bf458b783726bf53431b4633f2792f60748d9f74"
+  integrity sha1-v0WLeDcmv1NDG0Yz8nkvYHSNn3Q=
+  dependencies:
+    abs-svg-path "^0.1.1"
+    is-svg-path "^1.0.1"
+    normalize-svg-path "^1.0.0"
+    parse-svg-path "^0.1.2"
+
+svg-path-sdf@^1.1.3:
+  version "1.1.3"
+  resolved "https://registry.yarnpkg.com/svg-path-sdf/-/svg-path-sdf-1.1.3.tgz#92957a31784c0eaf68945472c8dc6bf9e6d126fc"
+  integrity sha512-vJJjVq/R5lSr2KLfVXVAStktfcfa1pNFjFOgyJnzZFXlO/fDZ5DmM8FpnSKKzLPfEYTVeXuVBTHF296TpxuJVg==
+  dependencies:
+    bitmap-sdf "^1.0.0"
+    draw-svg-path "^1.0.0"
+    is-svg-path "^1.0.1"
+    parse-svg-path "^0.1.2"
+    svg-path-bounds "^1.0.1"
+
 svgo@^1.0.0, svgo@^1.2.2:
   version "1.3.2"
   resolved "https://registry.yarnpkg.com/svgo/-/svgo-1.3.2.tgz#b6dc511c063346c9e415b81e43401145b96d4167"
@@ -12328,6 +15119,27 @@
   resolved "https://registry.yarnpkg.com/tapable/-/tapable-1.1.3.tgz#a1fccc06b58db61fd7a45da2da44f5f3a3e67ba2"
   integrity sha512-4WK/bYZmj8xLr+HUCODHGF1ZFzsYffasLUgEiMBY4fgtltdO6B4WJtlSbPaDTLpYTcGVwM2qLnFTICEcNxs3kA==
 
+tape@^4.0.0:
+  version "4.13.3"
+  resolved "https://registry.yarnpkg.com/tape/-/tape-4.13.3.tgz#51b3d91c83668c7a45b1a594b607dee0a0b46278"
+  integrity sha512-0/Y20PwRIUkQcTCSi4AASs+OANZZwqPKaipGCEwp10dQMipVvSZwUUCi01Y/OklIGyHKFhIcjock+DKnBfLAFw==
+  dependencies:
+    deep-equal "~1.1.1"
+    defined "~1.0.0"
+    dotignore "~0.1.2"
+    for-each "~0.3.3"
+    function-bind "~1.1.1"
+    glob "~7.1.6"
+    has "~1.0.3"
+    inherits "~2.0.4"
+    is-regex "~1.0.5"
+    minimist "~1.2.5"
+    object-inspect "~1.7.0"
+    resolve "~1.17.0"
+    resumer "~0.0.0"
+    string.prototype.trim "~1.2.1"
+    through "~2.3.8"
+
 term-size@^1.2.0:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/term-size/-/term-size-1.2.0.tgz#458b83887f288fc56d6fffbfad262e26638efa69"
@@ -12398,6 +15210,13 @@
     read-pkg-up "^4.0.0"
     require-main-filename "^2.0.0"
 
+text-cache@^4.2.2:
+  version "4.2.2"
+  resolved "https://registry.yarnpkg.com/text-cache/-/text-cache-4.2.2.tgz#d0d30ba89b7312ea1c1a31cd9a4db56c1cef7fe7"
+  integrity sha512-zky+UDYiX0a/aPw/YTBD+EzKMlCTu1chFuCMZeAkgoRiceySdROu1V2kJXhCbtEdBhiOviYnAdGiSYl58HW0ZQ==
+  dependencies:
+    vectorize-text "^3.2.1"
+
 text-table@0.2.0, text-table@^0.2.0:
   version "0.2.0"
   resolved "https://registry.yarnpkg.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4"
@@ -12408,7 +15227,15 @@
   resolved "https://registry.yarnpkg.com/throat/-/throat-4.1.0.tgz#89037cbc92c56ab18926e6ba4cbb200e15672a6a"
   integrity sha1-iQN8vJLFarGJJua6TLsgDhVnKmo=
 
-through2@^2.0.0:
+through2@^0.6.3:
+  version "0.6.5"
+  resolved "https://registry.yarnpkg.com/through2/-/through2-0.6.5.tgz#41ab9c67b29d57209071410e1d7a7a968cd3ad48"
+  integrity sha1-QaucZ7KdVyCQcUEOHXp6lozTrUg=
+  dependencies:
+    readable-stream ">=1.0.33-1 <1.1.0-0"
+    xtend ">=4.0.0 <4.1.0-0"
+
+through2@^2.0.0, through2@^2.0.1:
   version "2.0.5"
   resolved "https://registry.yarnpkg.com/through2/-/through2-2.0.5.tgz#01c1e39eb31d07cb7d03a96a70823260b23132cd"
   integrity sha512-/mrRod8xqpA+IHSLyGCQ2s8SPHiCDEeQJSep1jqLYeEUClOFG2Qsh+4FU6G9VeqpZnGW/Su8LQGc4YKni5rYSQ==
@@ -12416,7 +15243,15 @@
     readable-stream "~2.3.6"
     xtend "~4.0.1"
 
-through@^2.3.6:
+through2@~0.4.1:
+  version "0.4.2"
+  resolved "https://registry.yarnpkg.com/through2/-/through2-0.4.2.tgz#dbf5866031151ec8352bb6c4db64a2292a840b9b"
+  integrity sha1-2/WGYDEVHsg1K7bE22SiKSqEC5s=
+  dependencies:
+    readable-stream "~1.0.17"
+    xtend "~2.1.1"
+
+through@^2.3.6, through@~2.3.4, through@~2.3.8:
   version "2.3.8"
   resolved "https://registry.yarnpkg.com/through/-/through-2.3.8.tgz#0dd4c9ffaabc357960b1b724115d7e0e86a2e1f5"
   integrity sha1-DdTJ/6q8NXlgsbckEV1+Doai4fU=
@@ -12453,6 +15288,11 @@
   resolved "https://registry.yarnpkg.com/tinycolor2/-/tinycolor2-1.4.1.tgz#f4fad333447bc0b07d4dc8e9209d8f39a8ac77e8"
   integrity sha1-9PrTM0R7wLB9TcjpIJ2POaisd+g=
 
+tinyqueue@^2.0.3:
+  version "2.0.3"
+  resolved "https://registry.yarnpkg.com/tinyqueue/-/tinyqueue-2.0.3.tgz#64d8492ebf39e7801d7bd34062e29b45b2035f08"
+  integrity sha512-ppJZNDuKGgxzkHihX8v9v9G5f+18gzaTfrukGrq6ueg0lmH4nqVnA2IPG0AEH3jKEk2GRJCUhDoqpoiw3PHLBA==
+
 tmp@^0.0.33:
   version "0.0.33"
   resolved "https://registry.yarnpkg.com/tmp/-/tmp-0.0.33.tgz#6d34335889768d21b2bcda0aa277ced3b1bfadf9"
@@ -12473,6 +15313,15 @@
     is-absolute "^1.0.0"
     is-negated-glob "^1.0.0"
 
+to-array-buffer@^3.0.0:
+  version "3.2.0"
+  resolved "https://registry.yarnpkg.com/to-array-buffer/-/to-array-buffer-3.2.0.tgz#cb684dd691a7368c3b249c2348d75227f7d4dbb4"
+  integrity sha512-zN33mwi0gpL+7xW1ITLfJ48CEj6ZQW0ZAP0MU+2W3kEY0PAIncyuxmD4OqkUVhPAbTP7amq9j/iwvZKYS+lzSQ==
+  dependencies:
+    flatten-vertex-data "^1.0.2"
+    is-blob "^2.0.1"
+    string-to-arraybuffer "^1.0.0"
+
 to-arraybuffer@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/to-arraybuffer/-/to-arraybuffer-1.0.1.tgz#7d229b1fcc637e466ca081180836a7aabff83f43"
@@ -12483,6 +15332,11 @@
   resolved "https://registry.yarnpkg.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e"
   integrity sha1-3F5pjL0HkmW8c+A3doGk5Og/YW4=
 
+to-float32@^1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/to-float32/-/to-float32-1.0.1.tgz#22d5921f38183164b9e7e9876158c0c16cb9753a"
+  integrity sha512-nOy2WSwae3xhZbc+05xiCuU3ZPPmH0L4Rg4Q1qiOGFSuNSCTB9nVJaGgGl3ZScxAclX/L8hJuDHJGDAzbfuKCQ==
+
 to-object-path@^0.3.0:
   version "0.3.0"
   resolved "https://registry.yarnpkg.com/to-object-path/-/to-object-path-0.3.0.tgz#297588b7b0e7e0ac08e04e672f85c1f4999e17af"
@@ -12490,6 +15344,20 @@
   dependencies:
     kind-of "^3.0.2"
 
+to-px@1.0.1:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/to-px/-/to-px-1.0.1.tgz#5bbaed5e5d4f76445bcc903c293a2307dd324646"
+  integrity sha1-W7rtXl1PdkRbzJA8KTojB90yRkY=
+  dependencies:
+    parse-unit "^1.0.1"
+
+to-px@^1.0.1:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/to-px/-/to-px-1.1.0.tgz#b6b269ed5db0cc9aefc15272a4c8bcb2ca1e99ca"
+  integrity sha512-bfg3GLYrGoEzrGoE05TAL/Uw+H/qrf2ptr9V3W7U0lkjjyYnIfgxmVLUfhQ1hZpIQwin81uxhDjvUkDYsC0xWw==
+  dependencies:
+    parse-unit "^1.0.1"
+
 to-readable-stream@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/to-readable-stream/-/to-readable-stream-1.0.0.tgz#ce0aa0c2f3df6adf852efb404a783e77c0475771"
@@ -12520,6 +15388,17 @@
     regex-not "^1.0.2"
     safe-regex "^1.1.0"
 
+to-uint8@^1.4.1:
+  version "1.4.1"
+  resolved "https://registry.yarnpkg.com/to-uint8/-/to-uint8-1.4.1.tgz#9f45694905b827f247d37bc8ec83b2818d81fac9"
+  integrity sha512-o+ochsMlTZyucbww8It401FC2Rx+OP2RpDeYbA6h+y9HgedDl1UjdsJ9CmzKEG7AFP9es5PmJ4eDWeeeXihESg==
+  dependencies:
+    arr-flatten "^1.1.0"
+    clamp "^1.0.1"
+    is-base64 "^0.1.0"
+    is-float-array "^1.0.0"
+    to-array-buffer "^3.0.0"
+
 toggle-selection@^1.0.6:
   version "1.0.6"
   resolved "https://registry.yarnpkg.com/toggle-selection/-/toggle-selection-1.0.6.tgz#6e45b1263f2017fa0acc7d89d78b15b8bf77da32"
@@ -12530,6 +15409,13 @@
   resolved "https://registry.yarnpkg.com/toidentifier/-/toidentifier-1.0.0.tgz#7e1be3470f1e77948bc43d94a3c8f4d7752ba553"
   integrity sha512-yaOH/Pk/VEhBWWTlhI+qXxDFXlejDGcQipMlyxda9nthulaxLZUNcUqFxokp0vcYnvteJln5FNQDRrxj3YcbVw==
 
+topojson-client@^3.1.0:
+  version "3.1.0"
+  resolved "https://registry.yarnpkg.com/topojson-client/-/topojson-client-3.1.0.tgz#22e8b1ed08a2b922feeb4af6f53b6ef09a467b99"
+  integrity sha512-605uxS6bcYxGXw9qi62XyrV6Q3xwbndjachmNxu8HWTtVPxZfEJN9fd/SZS1Q54Sn2y0TMyMxFj/cJINqGHrKw==
+  dependencies:
+    commander "2"
+
 tough-cookie@^2.3.3, tough-cookie@^2.3.4, tough-cookie@^2.5.0:
   version "2.5.0"
   resolved "https://registry.yarnpkg.com/tough-cookie/-/tough-cookie-2.5.0.tgz#cd9fb2a0aa1d5a12b473bd9fb96fa3dcff65ade2"
@@ -12553,6 +15439,22 @@
   dependencies:
     punycode "^2.1.0"
 
+triangulate-hypercube@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/triangulate-hypercube/-/triangulate-hypercube-1.0.1.tgz#d8071db2ebfcfd51f308d0bcf2a5c48a5b36d137"
+  integrity sha1-2Acdsuv8/VHzCNC88qXEils20Tc=
+  dependencies:
+    gamma "^0.1.0"
+    permutation-parity "^1.0.0"
+    permutation-rank "^1.0.0"
+
+triangulate-polyline@^1.0.0:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/triangulate-polyline/-/triangulate-polyline-1.0.3.tgz#bf8ba877a85054103feb9fa5a61b4e8d7017814d"
+  integrity sha1-v4uod6hQVBA/65+lphtOjXAXgU0=
+  dependencies:
+    cdt2d "^1.0.0"
+
 trim-newlines@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/trim-newlines/-/trim-newlines-2.0.0.tgz#b403d0b91be50c331dfc4b82eeceb22c3de16d20"
@@ -12597,11 +15499,30 @@
   dependencies:
     safe-buffer "^5.0.1"
 
+turntable-camera-controller@^3.0.0:
+  version "3.0.1"
+  resolved "https://registry.yarnpkg.com/turntable-camera-controller/-/turntable-camera-controller-3.0.1.tgz#8dbd3fe00550191c65164cb888971049578afd99"
+  integrity sha1-jb0/4AVQGRxlFky4iJcQSVeK/Zk=
+  dependencies:
+    filtered-vector "^1.2.1"
+    gl-mat4 "^1.0.2"
+    gl-vec3 "^1.0.2"
+
 tweetnacl@^0.14.3, tweetnacl@~0.14.0:
   version "0.14.5"
   resolved "https://registry.yarnpkg.com/tweetnacl/-/tweetnacl-0.14.5.tgz#5ae68177f192d4456269d108afa93ff8743f4f64"
   integrity sha1-WuaBd/GS1EViadEIr6k/+HQ/T2Q=
 
+two-product@^1.0.0, two-product@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/two-product/-/two-product-1.0.2.tgz#67d95d4b257a921e2cb4bd7af9511f9088522eaa"
+  integrity sha1-Z9ldSyV6kh4stL16+VEfkIhSLqo=
+
+two-sum@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/two-sum/-/two-sum-1.0.0.tgz#31d3f32239e4f731eca9df9155e2b297f008ab64"
+  integrity sha1-MdPzIjnk9zHsqd+RVeKyl/AIq2Q=
+
 type-check@~0.3.2:
   version "0.3.2"
   resolved "https://registry.yarnpkg.com/type-check/-/type-check-0.3.2.tgz#5884cab512cf1d355e3fb784f30804b2b520db72"
@@ -12642,6 +15563,11 @@
     media-typer "0.3.0"
     mime-types "~2.1.24"
 
+type-name@^2.0.0:
+  version "2.0.2"
+  resolved "https://registry.yarnpkg.com/type-name/-/type-name-2.0.2.tgz#efe7d4123d8ac52afff7f40c7e4dec5266008fb4"
+  integrity sha1-7+fUEj2KxSr/9/QMfk3sUmYAj7Q=
+
 type@^1.0.1:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/type/-/type-1.2.0.tgz#848dd7698dafa3e54a6c479e759c4bc3f18847a0"
@@ -12652,6 +15578,14 @@
   resolved "https://registry.yarnpkg.com/type/-/type-2.0.0.tgz#5f16ff6ef2eb44f260494dae271033b29c09a9c3"
   integrity sha512-KBt58xCHry4Cejnc2ISQAF7QY+ORngsWfxezO68+12hKV6lQY8P/psIkcbjeHWn7MqcgciWJyCCevFMJdIXpow==
 
+typedarray-pool@^1.0.0, typedarray-pool@^1.0.2, typedarray-pool@^1.1.0:
+  version "1.2.0"
+  resolved "https://registry.yarnpkg.com/typedarray-pool/-/typedarray-pool-1.2.0.tgz#e7e90720144ba02b9ed660438af6f3aacfe33ac3"
+  integrity sha512-YTSQbzX43yvtpfRtIDAYygoYtgT+Rpjuxy9iOpczrjpXLgGoyG7aS5USJXV2d3nn8uHTeb9rXDvzS27zUg5KYQ==
+  dependencies:
+    bit-twiddle "^1.0.0"
+    dup "^1.0.0"
+
 typedarray-to-buffer@^3.1.5:
   version "3.1.5"
   resolved "https://registry.yarnpkg.com/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz#a97ee7a9ff42691b9f783ff1bc5112fe3fca9080"
@@ -12679,6 +15613,16 @@
   resolved "https://registry.yarnpkg.com/ua-parser-js/-/ua-parser-js-0.7.21.tgz#853cf9ce93f642f67174273cc34565ae6f308777"
   integrity sha512-+O8/qh/Qj8CgC6eYBVBykMrNtp5Gebn4dlGD/kKXVkJNDwyrAwSIqwz8CDf+tsAIWVycKcku6gIXJ0qwx/ZXaQ==
 
+uglify-js@^2.6.0:
+  version "2.8.29"
+  resolved "https://registry.yarnpkg.com/uglify-js/-/uglify-js-2.8.29.tgz#29c5733148057bb4e1f75df35b7a9cb72e6a59dd"
+  integrity sha1-KcVzMUgFe7Th913zW3qcty5qWd0=
+  dependencies:
+    source-map "~0.5.1"
+    yargs "~3.10.0"
+  optionalDependencies:
+    uglify-to-browserify "~1.0.0"
+
 uglify-js@^3.1.4:
   version "3.7.4"
   resolved "https://registry.yarnpkg.com/uglify-js/-/uglify-js-3.7.4.tgz#e6d83a1aa32ff448bd1679359ab13d8db0fe0743"
@@ -12687,6 +15631,11 @@
     commander "~2.20.3"
     source-map "~0.6.1"
 
+uglify-to-browserify@~1.0.0:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/uglify-to-browserify/-/uglify-to-browserify-1.0.2.tgz#6e0924d6bda6b5afe349e39a6d632850a0f882b7"
+  integrity sha1-bgkk1r2mta/jSeOabWMoUKD4grc=
+
 unc-path-regex@^0.1.2:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/unc-path-regex/-/unc-path-regex-0.1.2.tgz#e73dd3d7b0d7c5ed86fbac6b0ae7d8c6a69d50fa"
@@ -12715,6 +15664,16 @@
   resolved "https://registry.yarnpkg.com/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-1.0.5.tgz#a9cc6cc7ce63a0a3023fc99e341b94431d405a57"
   integrity sha512-L5RAqCfXqAwR3RriF8pM0lU0w4Ryf/GgzONwi6KnL1taJQa7x1TCxdJnILX59WIGOwR57IVxn7Nej0fz1Ny6fw==
 
+union-find@^1.0.0, union-find@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/union-find/-/union-find-1.0.2.tgz#292bac415e6ad3a89535d237010db4a536284e58"
+  integrity sha1-KSusQV5q06iVNdI3AQ20pTYoTlg=
+
+union-find@~0.0.3:
+  version "0.0.4"
+  resolved "https://registry.yarnpkg.com/union-find/-/union-find-0.0.4.tgz#b854b3301619bdad144b0014c78f96eac0d2f0f6"
+  integrity sha1-uFSzMBYZva0USwAUx4+W6sDS8PY=
+
 union-value@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/union-value/-/union-value-1.0.1.tgz#0b6fe7b835aecda61c6ea4d4f02c14221e109847"
@@ -12725,7 +15684,7 @@
     is-extendable "^0.1.1"
     set-value "^2.0.1"
 
-uniq@^1.0.1:
+uniq@^1.0.0, uniq@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/uniq/-/uniq-1.0.1.tgz#b31c5ae8254844a3a8281541ce2b04b865a734ff"
   integrity sha1-sxxa6CVIRKOoKBVBzisEuGWnNP8=
@@ -12778,7 +15737,7 @@
   resolved "https://registry.yarnpkg.com/unpipe/-/unpipe-1.0.0.tgz#b2bf4ee8514aae6165b4817829d21b2ef49904ec"
   integrity sha1-sr9O6FFKrmFltIF4KdIbLvSZBOw=
 
-unquote@~1.1.1:
+unquote@^1.1.0, unquote@~1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/unquote/-/unquote-1.1.1.tgz#8fded7324ec6e88a0ff8b905e7c098cdc086d544"
   integrity sha1-j97XMk7G6IoP+LkF58CYzcCG1UQ=
@@ -12796,6 +15755,11 @@
   resolved "https://registry.yarnpkg.com/upath/-/upath-1.2.0.tgz#8f66dbcd55a883acdae4408af8b035a5044c1894"
   integrity sha512-aZwGpamFO61g3OlfT7OQCHqhGnW43ieH9WZeP7QxN/G/jS4jfqUkZxoryvJgVPEcrl5NL/ggHsSmLMHuH64Lhg==
 
+update-diff@^1.1.0:
+  version "1.1.0"
+  resolved "https://registry.yarnpkg.com/update-diff/-/update-diff-1.1.0.tgz#f510182d81ee819fb82c3a6b22b62bbdeda7808f"
+  integrity sha1-9RAYLYHugZ+4LDprIrYrve2ngI8=
+
 update-notifier@^3.0.1:
   version "3.0.1"
   resolved "https://registry.yarnpkg.com/update-notifier/-/update-notifier-3.0.1.tgz#78ecb68b915e2fd1be9f767f6e298ce87b736250"
@@ -12914,11 +15878,50 @@
   resolved "https://registry.yarnpkg.com/utila/-/utila-0.4.0.tgz#8a16a05d445657a3aea5eecc5b12a4fa5379772c"
   integrity sha1-ihagXURWV6Oupe7MWxKk+lN5dyw=
 
+utils-copy-error@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/utils-copy-error/-/utils-copy-error-1.0.1.tgz#791de393c0f09890afd59f3cbea635f079a94fa5"
+  integrity sha1-eR3jk8DwmJCv1Z88vqY18HmpT6U=
+  dependencies:
+    object-keys "^1.0.9"
+    utils-copy "^1.1.0"
+
+utils-copy@^1.0.0, utils-copy@^1.1.0:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/utils-copy/-/utils-copy-1.1.1.tgz#6e2b97982aa8cd73e1182a3e6f8bec3c0f4058a7"
+  integrity sha1-biuXmCqozXPhGCo+b4vsPA9AWKc=
+  dependencies:
+    const-pinf-float64 "^1.0.0"
+    object-keys "^1.0.9"
+    type-name "^2.0.0"
+    utils-copy-error "^1.0.0"
+    utils-indexof "^1.0.0"
+    utils-regex-from-string "^1.0.0"
+    validate.io-array "^1.0.3"
+    validate.io-buffer "^1.0.1"
+    validate.io-nonnegative-integer "^1.0.0"
+
+utils-indexof@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/utils-indexof/-/utils-indexof-1.0.0.tgz#20feabf09ef1018b523643e8380e7bc83ec61b5c"
+  integrity sha1-IP6r8J7xAYtSNkPoOA57yD7GG1w=
+  dependencies:
+    validate.io-array-like "^1.0.1"
+    validate.io-integer-primitive "^1.0.0"
+
 utils-merge@1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/utils-merge/-/utils-merge-1.0.1.tgz#9f95710f50a267947b2ccc124741c1028427e713"
   integrity sha1-n5VxD1CiZ5R7LMwSR0HBAoQn5xM=
 
+utils-regex-from-string@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/utils-regex-from-string/-/utils-regex-from-string-1.0.0.tgz#fe1a2909f8de0ff0d5182c80fbc654d6a687d189"
+  integrity sha1-/hopCfjeD/DVGCyA+8ZU1qaH0Yk=
+  dependencies:
+    regex-regex "^1.0.0"
+    validate.io-string-primitive "^1.0.0"
+
 uuid@^3.0.1, uuid@^3.3.2:
   version "3.3.3"
   resolved "https://registry.yarnpkg.com/uuid/-/uuid-3.3.3.tgz#4568f0216e78760ee1dbf3a4d2cf53e224112866"
@@ -12937,6 +15940,77 @@
     spdx-correct "^3.0.0"
     spdx-expression-parse "^3.0.0"
 
+validate.io-array-like@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/validate.io-array-like/-/validate.io-array-like-1.0.2.tgz#7af9f7eb7b51715beb2215668ec5cce54faddb5a"
+  integrity sha1-evn363tRcVvrIhVmjsXM5U+t21o=
+  dependencies:
+    const-max-uint32 "^1.0.2"
+    validate.io-integer-primitive "^1.0.0"
+
+validate.io-array@^1.0.3, validate.io-array@^1.0.6:
+  version "1.0.6"
+  resolved "https://registry.yarnpkg.com/validate.io-array/-/validate.io-array-1.0.6.tgz#5b5a2cafd8f8b85abb2f886ba153f2d93a27774d"
+  integrity sha1-W1osr9j4uFq7L4hroVPy2Tond00=
+
+validate.io-buffer@^1.0.1:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/validate.io-buffer/-/validate.io-buffer-1.0.2.tgz#852d6734021914d5d13afc32531761e3720ed44e"
+  integrity sha1-hS1nNAIZFNXROvwyUxdh43IO1E4=
+
+validate.io-integer-primitive@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/validate.io-integer-primitive/-/validate.io-integer-primitive-1.0.0.tgz#a9aa010355fe8681c0fea6c1a74ad2419cadddc6"
+  integrity sha1-qaoBA1X+hoHA/qbBp0rSQZyt3cY=
+  dependencies:
+    validate.io-number-primitive "^1.0.0"
+
+validate.io-integer@^1.0.5:
+  version "1.0.5"
+  resolved "https://registry.yarnpkg.com/validate.io-integer/-/validate.io-integer-1.0.5.tgz#168496480b95be2247ec443f2233de4f89878068"
+  integrity sha1-FoSWSAuVviJH7EQ/IjPeT4mHgGg=
+  dependencies:
+    validate.io-number "^1.0.3"
+
+validate.io-matrix-like@^1.0.2:
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/validate.io-matrix-like/-/validate.io-matrix-like-1.0.2.tgz#5ec32a75d0889dac736dea68bdd6145b155edfc3"
+  integrity sha1-XsMqddCInaxzbepovdYUWxVe38M=
+
+validate.io-ndarray-like@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/validate.io-ndarray-like/-/validate.io-ndarray-like-1.0.0.tgz#d8a3b0ed165bbf1d2fc0d0073270cfa552295919"
+  integrity sha1-2KOw7RZbvx0vwNAHMnDPpVIpWRk=
+
+validate.io-nonnegative-integer@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/validate.io-nonnegative-integer/-/validate.io-nonnegative-integer-1.0.0.tgz#8069243a08c5f98e95413c929dfd7b18f3f6f29f"
+  integrity sha1-gGkkOgjF+Y6VQTySnf17GPP28p8=
+  dependencies:
+    validate.io-integer "^1.0.5"
+
+validate.io-number-primitive@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/validate.io-number-primitive/-/validate.io-number-primitive-1.0.0.tgz#d2e01f202989369dcf1155449564203afe584e55"
+  integrity sha1-0uAfICmJNp3PEVVElWQgOv5YTlU=
+
+validate.io-number@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/validate.io-number/-/validate.io-number-1.0.3.tgz#f63ffeda248bf28a67a8d48e0e3b461a1665baf8"
+  integrity sha1-9j/+2iSL8opnqNSODjtGGhZluvg=
+
+validate.io-positive-integer@^1.0.0:
+  version "1.0.0"
+  resolved "https://registry.yarnpkg.com/validate.io-positive-integer/-/validate.io-positive-integer-1.0.0.tgz#7ed2d03b4c27558cc66a00aab0f0e921814a6582"
+  integrity sha1-ftLQO0wnVYzGagCqsPDpIYFKZYI=
+  dependencies:
+    validate.io-integer "^1.0.5"
+
+validate.io-string-primitive@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/validate.io-string-primitive/-/validate.io-string-primitive-1.0.1.tgz#b8135b9fb1372bde02fdd53ad1d0ccd6de798fee"
+  integrity sha1-uBNbn7E3K94C/dU60dDM1t55j+4=
+
 value-equal@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/value-equal/-/value-equal-1.0.1.tgz#1e0b794c734c5c0cade179c437d356d931a34d6c"
@@ -12947,6 +16021,19 @@
   resolved "https://registry.yarnpkg.com/vary/-/vary-1.1.2.tgz#2299f02c6ded30d4a5961b0b9f74524a18f634fc"
   integrity sha1-IpnwLG3tMNSllhsLn3RSShj2NPw=
 
+vectorize-text@^3.2.1:
+  version "3.2.1"
+  resolved "https://registry.yarnpkg.com/vectorize-text/-/vectorize-text-3.2.1.tgz#85921abd9685af775fd20a01041a2837fe51bdb5"
+  integrity sha512-rGojF+D9BB96iPZPUitfq5kaiS6eCJmfEel0NXOK/MzZSuXGiwhoop80PtaDas9/Hg/oaox1tI9g3h93qpuspg==
+  dependencies:
+    cdt2d "^1.0.0"
+    clean-pslg "^1.1.0"
+    ndarray "^1.0.11"
+    planar-graph-to-polyline "^1.0.0"
+    simplify-planar-graph "^2.0.1"
+    surface-nets "^1.0.0"
+    triangulate-polyline "^1.0.0"
+
 vendors@^1.0.0:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/vendors/-/vendors-1.0.3.tgz#a6467781abd366217c050f8202e7e50cc9eef8c0"
@@ -12966,6 +16053,15 @@
   resolved "https://registry.yarnpkg.com/vm-browserify/-/vm-browserify-1.1.2.tgz#78641c488b8e6ca91a75f511e7a3b32a86e5dda0"
   integrity sha512-2ham8XPWTONajOR0ohOKOHXkm3+gaBmGut3SRuu75xLd/RRaY6vqgh8NBYYk7+RW3u5AtzPQZG8F10LHkl0lAQ==
 
+vt-pbf@^3.1.1:
+  version "3.1.1"
+  resolved "https://registry.yarnpkg.com/vt-pbf/-/vt-pbf-3.1.1.tgz#b0f627e39a10ce91d943b898ed2363d21899fb82"
+  integrity sha512-pHjWdrIoxurpmTcbfBWXaPwSmtPAHS105253P1qyEfSTV2HJddqjM+kIHquaT/L6lVJIk9ltTGc0IxR/G47hYA==
+  dependencies:
+    "@mapbox/point-geometry" "0.1.0"
+    "@mapbox/vector-tile" "^1.3.1"
+    pbf "^3.0.5"
+
 w3c-hr-time@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/w3c-hr-time/-/w3c-hr-time-1.0.1.tgz#82ac2bff63d950ea9e3189a58a65625fedf19045"
@@ -13012,6 +16108,23 @@
   dependencies:
     minimalistic-assert "^1.0.0"
 
+weak-map@^1.0.5:
+  version "1.0.5"
+  resolved "https://registry.yarnpkg.com/weak-map/-/weak-map-1.0.5.tgz#79691584d98607f5070bd3b70a40e6bb22e401eb"
+  integrity sha1-eWkVhNmGB/UHC9O3CkDmuyLkAes=
+
+weakmap-shim@^1.1.0:
+  version "1.1.1"
+  resolved "https://registry.yarnpkg.com/weakmap-shim/-/weakmap-shim-1.1.1.tgz#d65afd784109b2166e00ff571c33150ec2a40b49"
+  integrity sha1-1lr9eEEJshZuAP9XHDMVDsKkC0k=
+
+webgl-context@^2.2.0:
+  version "2.2.0"
+  resolved "https://registry.yarnpkg.com/webgl-context/-/webgl-context-2.2.0.tgz#8f37d7257cf6df1cd0a49e6a7b1b721b94cc86a0"
+  integrity sha1-jzfXJXz23xzQpJ5qextyG5TMhqA=
+  dependencies:
+    get-canvas-context "^1.0.1"
+
 webidl-conversions@^4.0.2:
   version "4.0.2"
   resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-4.0.2.tgz#a855980b1f0b6b359ba1d5d9fb39ae941faa63ad"
@@ -13204,11 +16317,21 @@
   dependencies:
     string-width "^4.0.0"
 
+window-size@0.1.0:
+  version "0.1.0"
+  resolved "https://registry.yarnpkg.com/window-size/-/window-size-0.1.0.tgz#5438cd2ea93b202efa3a19fe8887aee7c94f9c9d"
+  integrity sha1-VDjNLqk7IC76Ohn+iIeu58lPnJ0=
+
 word-wrap@~1.2.3:
   version "1.2.3"
   resolved "https://registry.yarnpkg.com/word-wrap/-/word-wrap-1.2.3.tgz#610636f6b1f703891bd34771ccb17fb93b47079c"
   integrity sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ==
 
+wordwrap@0.0.2:
+  version "0.0.2"
+  resolved "https://registry.yarnpkg.com/wordwrap/-/wordwrap-0.0.2.tgz#b79669bb42ecb409f83d583cad52ca17eaa1643f"
+  integrity sha1-t5Zpu0LstAn4PVg8rVLKF+qhZD8=
+
 wordwrap@~0.0.2:
   version "0.0.3"
   resolved "https://registry.yarnpkg.com/wordwrap/-/wordwrap-0.0.3.tgz#a3d5da6cd5c0bc0008d37234bbaf1bed63059107"
@@ -13363,6 +16486,13 @@
   dependencies:
     microevent.ts "~0.1.1"
 
+world-calendars@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/world-calendars/-/world-calendars-1.0.3.tgz#b25c5032ba24128ffc41d09faf4a5ec1b9c14335"
+  integrity sha1-slxQMrokEo/8QdCfr0pewbnBQzU=
+  dependencies:
+    object-assign "^4.1.0"
+
 wrap-ansi@^2.0.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-2.1.0.tgz#d8fc3d284dd05794fe84973caecdd1cf824fdd85"
@@ -13510,11 +16640,23 @@
   dependencies:
     "@babel/runtime-corejs3" "^7.8.3"
 
-xtend@^4.0.0, xtend@~4.0.1:
+"xtend@>=4.0.0 <4.1.0-0", xtend@^4.0.0, xtend@~4.0.1:
   version "4.0.2"
   resolved "https://registry.yarnpkg.com/xtend/-/xtend-4.0.2.tgz#bb72779f5fa465186b1f438f674fa347fdb5db54"
   integrity sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ==
 
+xtend@^2.1.2:
+  version "2.2.0"
+  resolved "https://registry.yarnpkg.com/xtend/-/xtend-2.2.0.tgz#eef6b1f198c1c8deafad8b1765a04dad4a01c5a9"
+  integrity sha1-7vax8ZjByN6vrYsXZaBNrUoBxak=
+
+xtend@~2.1.1:
+  version "2.1.2"
+  resolved "https://registry.yarnpkg.com/xtend/-/xtend-2.1.2.tgz#6efecc2a4dad8e6962c4901b337ce7ba87b5d28b"
+  integrity sha1-bv7MKk2tjmlixJAbM3znuoe10os=
+  dependencies:
+    object-keys "~0.4.0"
+
 "y18n@^3.2.1 || ^4.0.0", y18n@^4.0.0:
   version "4.0.0"
   resolved "https://registry.yarnpkg.com/y18n/-/y18n-4.0.0.tgz#95ef94f85ecc81d007c264e190a120f0a3c8566b"
@@ -13623,3 +16765,20 @@
     which-module "^2.0.0"
     y18n "^4.0.0"
     yargs-parser "^15.0.0"
+
+yargs@~3.10.0:
+  version "3.10.0"
+  resolved "https://registry.yarnpkg.com/yargs/-/yargs-3.10.0.tgz#f7ee7bd857dd7c1d2d38c0e74efbd681d1431fd1"
+  integrity sha1-9+572FfdfB0tOMDnTvvWgdFDH9E=
+  dependencies:
+    camelcase "^1.0.2"
+    cliui "^2.1.0"
+    decamelize "^1.0.0"
+    window-size "0.1.0"
+
+zero-crossings@^1.0.0:
+  version "1.0.1"
+  resolved "https://registry.yarnpkg.com/zero-crossings/-/zero-crossings-1.0.1.tgz#c562bd3113643f3443a245d12406b88b69b9a9ff"
+  integrity sha1-xWK9MRNkPzRDokXRJAa4i2m5qf8=
+  dependencies:
+    cwise-compiler "^1.0.0"
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java
new file mode 100644
index 0000000..f194b29
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java
@@ -0,0 +1,145 @@
+/*
+ * 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.hadoop.ozone.debug;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.rocksdb.*;
+import picocli.CommandLine;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+/**
+ * Parser for scm.db file.
+ */
+@CommandLine.Command(
+        name = "scan",
+        description = "Parse specified metadataTable"
+)
+public class DBScanner implements Callable<Void> {
+
+  @CommandLine.Option(names = {"--column_family"},
+            description = "Table name")
+  private String tableName;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  private HashMap<String, DBColumnFamilyDefinition> columnFamilyMap;
+
+  private static void displayTable(RocksDB rocksDB,
+        DBColumnFamilyDefinition dbColumnFamilyDefinition,
+        List<ColumnFamilyHandle> list) throws IOException {
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+            dbColumnFamilyDefinition.getTableName()
+                    .getBytes(StandardCharsets.UTF_8), list);
+    if (columnFamilyHandle==null){
+      throw new IllegalArgumentException("columnFamilyHandle is null");
+    }
+    RocksIterator iterator = rocksDB.newIterator(columnFamilyHandle);
+    iterator.seekToFirst();
+    while (iterator.isValid()){
+      Object o = dbColumnFamilyDefinition.getValueCodec()
+              .fromPersistedFormat(iterator.value());
+      Gson gson = new GsonBuilder().setPrettyPrinting().create();
+      String result = gson.toJson(o);
+      System.out.println(result);
+      iterator.next();
+    }
+  }
+
+  private static ColumnFamilyHandle getColumnFamilyHandle(
+            byte[] name, List<ColumnFamilyHandle> columnFamilyHandles) {
+    return columnFamilyHandles
+            .stream()
+            .filter(
+              handle -> {
+                try {
+                  return Arrays.equals(handle.getName(), name);
+                    } catch (Exception ex) {
+                  throw new RuntimeException(ex);
+                    }
+              })
+            .findAny()
+            .orElse(null);
+  }
+
+  private void constructColumnFamilyMap(DBDefinition dbDefinition) {
+    this.columnFamilyMap = new HashMap<>();
+    DBColumnFamilyDefinition[] columnFamilyDefinitions = dbDefinition
+            .getColumnFamilies();
+    for(DBColumnFamilyDefinition definition:columnFamilyDefinitions){
+      this.columnFamilyMap.put(definition.getTableName(), definition);
+    }
+  }
+
+  @Override
+  public Void call() throws Exception {
+    List<ColumnFamilyDescriptor> cfs = new ArrayList<>();
+    final List<ColumnFamilyHandle> columnFamilyHandleList =
+            new ArrayList<>();
+    List<byte[]> cfList = null;
+    cfList = RocksDB.listColumnFamilies(new Options(),
+            parent.getDbPath());
+    if (cfList != null) {
+      for (byte[] b : cfList) {
+        cfs.add(new ColumnFamilyDescriptor(b));
+      }
+    }
+    RocksDB rocksDB = null;
+    rocksDB = RocksDB.openReadOnly(parent.getDbPath(),
+            cfs, columnFamilyHandleList);
+    this.printAppropriateTable(columnFamilyHandleList,
+           rocksDB, parent.getDbPath());
+    return null;
+  }
+
+  private void printAppropriateTable(
+          List<ColumnFamilyHandle> columnFamilyHandleList,
+          RocksDB rocksDB, String dbPath) throws IOException {
+    dbPath = removeTrailingSlashIfNeeded(dbPath);
+    if (dbPath.endsWith(new SCMDBDefinition().getName())){
+      this.constructColumnFamilyMap(new SCMDBDefinition());
+    }
+    if (this.columnFamilyMap !=null) {
+      if (!this.columnFamilyMap.containsKey(tableName)) {
+        System.out.print("Table with specified name does not exist");
+      } else {
+        DBColumnFamilyDefinition columnFamilyDefinition =
+                this.columnFamilyMap.get(tableName);
+        displayTable(rocksDB, columnFamilyDefinition, columnFamilyHandleList);
+      }
+    } else {
+      System.out.println("Incorrect db Path");
+    }
+  }
+
+  private String removeTrailingSlashIfNeeded(String dbPath) {
+    if(dbPath.endsWith(OzoneConsts.OZONE_URI_DELIMITER)){
+      dbPath = dbPath.substring(0, dbPath.length()-1);
+    }
+    return dbPath;
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ListTables.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ListTables.java
new file mode 100644
index 0000000..5aa5ed2
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ListTables.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hadoop.ozone.debug;
+
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import picocli.CommandLine;
+
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * List all column Families/Tables in db.
+ */
+@CommandLine.Command(
+        name = "list_column_families",
+        aliases = "ls",
+        description = "list all column families in db."
+)
+public class ListTables implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @Override
+  public Void call() throws Exception {
+    List<byte[]> columnFamilies = RocksDB.listColumnFamilies(new Options(),
+            parent.getDbPath());
+    for (byte[] b : columnFamilies) {
+      System.out.println(new String(b, StandardCharsets.UTF_8));
+    }
+    return null;
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
index 25295f7..82808d6 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
@@ -31,7 +31,8 @@
         versionProvider = HddsVersionProvider.class,
         subcommands = {
                 ChunkKeyHandler.class,
-                RatisLogParser.class
+                RatisLogParser.class,
+                RDBParser.class
         },
         mixinStandardHelpOptions = true)
 public class OzoneDebug extends GenericCli {
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/RDBParser.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/RDBParser.java
new file mode 100644
index 0000000..ae82ba1
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/RDBParser.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hadoop.ozone.debug;
+
+import org.apache.hadoop.hdds.cli.GenericCli;
+import picocli.CommandLine;
+
+/**
+ * Tool that parses rocksdb file.
+ */
+@CommandLine.Command(
+        name = "ldb",
+        description = "Parse rocksdb file content",
+        subcommands = {
+                DBScanner.class,
+                ListTables.class
+        })
+public class RDBParser extends GenericCli {
+
+  @CommandLine.Option(names = {"--db"},
+            description = "Database File Path")
+    private  String dbPath;
+
+  public String getDbPath() {
+    return dbPath;
+  }
+
+  @Override
+  public void execute(String[] argv) {
+    new RDBParser().run(argv);
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
index 8c37659..9cb22f6 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
@@ -111,7 +111,7 @@
         .setFactor(factor)
         .setKeyName(generateObjectName(counter))
         .setLocationInfoList(new ArrayList<>())
-        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroups(),
+        .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
             ALL, ALL))
         .build();