diff --git a/.gitignore b/.gitignore index 655de677ca..03d0593ed8 100644 --- a/.gitignore +++ b/.gitignore @@ -43,3 +43,8 @@ nohup.out # chunjun plugins chunjun-dist/ + +*/.cache +*/node_modules +/website/public +/website/public/page-data/ \ No newline at end of file diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java index 226ecaa090..7d2ac6609c 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java @@ -28,7 +28,7 @@ import java.util.Map; /** - * @program: flinkx + * @program: chunjun * @author: wuren * @create: 2021/10/19 */ diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java index cd8697f91c..48340daf06 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java @@ -19,7 +19,7 @@ package com.dtstack.chunjun.connector.hbase; /** - * @program: flinkx + * @program: chunjun * @author: wuren * @create: 2021/10/15 */ diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java index 5f1ba7cede..52c2887a49 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java @@ -40,7 +40,7 @@ /** * @author wuren - * @program flinkx + * @program chunjun * @create 2021/04/30 */ diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java b/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java index 59a889b7d0..b95ae97ced 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java @@ -50,6 +50,10 @@ import com.dtstack.chunjun.util.PropertiesUtil; import com.dtstack.chunjun.util.TableUtil; +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; + import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.execution.JobClient; @@ -69,9 +73,6 @@ import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.types.DataType; -import com.google.common.base.Preconditions; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,6 +139,7 @@ public static void main(String[] args) throws Exception { * @param tableEnv * @param job * @param options + * * @throws Exception */ private static void exeSqlJob( @@ -171,6 +173,7 @@ private static void exeSqlJob( * @param tableEnv * @param job * @param options + * * @throws Exception */ private static void exeSyncJob( @@ -179,7 +182,7 @@ private static void exeSyncJob( String job, Options options) throws Exception { - SyncConf config = parseFlinkxConf(job, options); + SyncConf config = parseConf(job, options); configStreamExecutionEnvironment(env, options, config); SourceFactory sourceFactory = DataSyncFactoryUtil.discoverSource(config, env); @@ -236,6 +239,7 @@ private static void exeSyncJob( * @param tableEnv * @param config * @param sourceDataStream + * * @return */ private static DataStream syncStreamToTable( @@ -273,17 +277,14 @@ private static DataStream syncStreamToTable( * * @param job * @param options + * * @return */ - public static SyncConf parseFlinkxConf(String job, Options options) { + public static SyncConf parseConf(String job, Options options) { SyncConf config; try { config = SyncConf.parseJob(job); - if (StringUtils.isNotBlank(options.getFlinkxDistDir())) { - config.setPluginRoot(options.getFlinkxDistDir()); - } - Properties confProperties = PropertiesUtil.parseConf(options.getConfProp()); String savePointPath = @@ -292,9 +293,6 @@ public static SyncConf parseFlinkxConf(String job, Options options) { config.setSavePointPath(savePointPath); } - if (StringUtils.isNotBlank(options.getRemoteFlinkxDistDir())) { - config.setRemotePluginPath(options.getRemoteFlinkxDistDir()); - } } catch (Exception e) { throw new ChunJunRuntimeException(e); } @@ -306,7 +304,7 @@ public static SyncConf parseFlinkxConf(String job, Options options) { * * @param env StreamExecutionEnvironment * @param options options - * @param config FlinkxConf + * @param config ChunJunConf */ private static void configStreamExecutionEnvironment( StreamExecutionEnvironment env, Options options, SyncConf config) { @@ -317,13 +315,13 @@ private static void configStreamExecutionEnvironment( } else { Preconditions.checkArgument( ExecuteProcessHelper.checkRemoteSqlPluginPath( - options.getRemoteFlinkxDistDir(), + options.getRemoteChunJunDistDir(), options.getMode(), options.getPluginLoadMode()), "Non-local mode or shipfile deployment mode, remoteSqlPluginPath is required"); FactoryHelper factoryHelper = new FactoryHelper(); - factoryHelper.setLocalPluginPath(options.getFlinkxDistDir()); - factoryHelper.setRemotePluginPath(options.getRemoteFlinkxDistDir()); + factoryHelper.setLocalPluginPath(options.getChunjunDistDir()); + factoryHelper.setRemotePluginPath(options.getRemoteChunJunDistDir()); factoryHelper.setPluginLoadMode(options.getPluginLoadMode()); factoryHelper.setEnv(env); factoryHelper.setExecutionMode(options.getMode()); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/conf/ChunJunCommonConf.java b/chunjun-core/src/main/java/com/dtstack/chunjun/conf/ChunJunCommonConf.java index a36f697386..0048cad8ac 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/conf/ChunJunCommonConf.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/conf/ChunJunCommonConf.java @@ -218,7 +218,7 @@ public void setExecuteDdlAble(boolean executeDdlAble) { @Override public String toString() { - return "FlinkxCommonConf{" + return "ChunJunCommonConf{" + "speedBytes=" + speedBytes + ", errorRecord=" diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/constants/Metrics.java b/chunjun-core/src/main/java/com/dtstack/chunjun/constants/Metrics.java index 13c7c8afb5..010544e95d 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/constants/Metrics.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/constants/Metrics.java @@ -66,7 +66,7 @@ public class Metrics { public static final String MAX_VALUE = "maxValue"; - public static final String METRIC_GROUP_KEY_FLINKX = "flinkx"; + public static final String METRIC_GROUP_KEY_CHUNJUN = "chunjun"; public static final String METRIC_GROUP_VALUE_OUTPUT = "output"; diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java index b4fa29d7e6..bacc0f397f 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java @@ -52,7 +52,7 @@ public BaseMetric(RuntimeContext runtimeContext) { runtimeContext .getMetricGroup() .addGroup( - Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_OUTPUT); + Metrics.METRIC_GROUP_KEY_CHUNJUN, Metrics.METRIC_GROUP_VALUE_OUTPUT); chunjunDirtyMetricGroup = chunjunMetricGroup.addGroup( diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/options/Options.java b/chunjun-core/src/main/java/com/dtstack/chunjun/options/Options.java index 632b827e32..b31e531c1b 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/options/Options.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/options/Options.java @@ -22,13 +22,14 @@ import com.dtstack.chunjun.enums.ClusterMode; import com.dtstack.chunjun.util.PropertiesUtil; +import org.apache.commons.lang.StringUtils; + import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,10 +59,6 @@ public class Options { @OptionRequired(description = "Flink configuration directory") private String flinkConfDir; - @Deprecated - @OptionRequired(description = "ChunJun dist dir") - private String flinkxDistDir; - @OptionRequired(description = "ChunJun dist dir") private String chunjunDistDir; @@ -80,10 +77,6 @@ public class Options { @OptionRequired(description = "plugin load mode, by classpath or shipfile") private String pluginLoadMode = "shipfile"; - @Deprecated - @OptionRequired(description = "remote ChunJun dist dir") - private String remoteFlinkxDistDir; - @OptionRequired(description = "remote ChunJun dist dir") private String remoteChunJunDistDir; @@ -145,17 +138,7 @@ public void setFlinkConfDir(String flinkConfDir) { } public String getChunjunDistDir() { - String flinkxDistDir = this.flinkxDistDir; - String chunjunDistDir = this.chunjunDistDir; - String distDir; - - if (StringUtils.isNotBlank(flinkxDistDir)) { - LOG.warn("Option 'flinkxDistDir' is deprecated, please replace with 'chunjunDistDir'."); - distDir = flinkxDistDir; - } else { - distDir = chunjunDistDir; - } - return distDir; + return this.chunjunDistDir; } public void setChunjunDistDir(String chunjunDistDir) { @@ -203,18 +186,7 @@ public void setPluginLoadMode(String pluginLoadMode) { } public String getRemoteChunJunDistDir() { - String remoteFlinkxDistDir = this.remoteFlinkxDistDir; - String remoteChunJunDistDir = this.remoteChunJunDistDir; - String remoteDir; - - if (StringUtils.isNotBlank(remoteFlinkxDistDir)) { - LOG.warn( - "Option 'remoteFlinkxDistDir' is deprecated, please replace with 'remoteChunJunDistDir'."); - remoteDir = remoteFlinkxDistDir; - } else { - remoteDir = remoteChunJunDistDir; - } - return remoteDir; + return this.remoteChunJunDistDir; } public void setRemoteChunJunDistDir(String remoteChunJunDistDir) { @@ -253,43 +225,6 @@ public void setJobType(String jobType) { this.jobType = jobType; } - public String getFlinkxDistDir() { - String flinkxDistDir = this.flinkxDistDir; - String chunjunDistDir = this.chunjunDistDir; - String distDir; - - if (StringUtils.isNotBlank(flinkxDistDir)) { - LOG.warn("Option 'flinkxDistDir' is deprecated, please replace with 'chunjunDistDir'."); - distDir = flinkxDistDir; - } else { - distDir = chunjunDistDir; - } - return distDir; - } - - public void setFlinkxDistDir(String flinkxDistDir) { - this.flinkxDistDir = flinkxDistDir; - } - - public String getRemoteFlinkxDistDir() { - String remoteFlinkxDistDir = this.remoteFlinkxDistDir; - String remoteChunJunDistDir = this.remoteChunJunDistDir; - String remoteDir; - - if (StringUtils.isNotBlank(remoteFlinkxDistDir)) { - LOG.warn( - "Option 'remoteFlinkxDistDir' is deprecated, please replace with 'remoteChunJunDistDir'."); - remoteDir = remoteFlinkxDistDir; - } else { - remoteDir = remoteChunJunDistDir; - } - return remoteDir; - } - - public void setRemoteFlinkxDistDir(String remoteFlinkxDistDir) { - this.remoteFlinkxDistDir = remoteFlinkxDistDir; - } - @Override public String toString() { return new StringJoiner(", ", Options.class.getSimpleName() + "[", "]") diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/sink/SinkFactory.java b/chunjun-core/src/main/java/com/dtstack/chunjun/sink/SinkFactory.java index ea3e6da978..fb87ebe3dc 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/sink/SinkFactory.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/sink/SinkFactory.java @@ -88,7 +88,7 @@ protected DataStreamSink createOutput( return createOutput(dataSet, outputFormat, this.getClass().getSimpleName().toLowerCase()); } - /** 初始化FlinkxCommonConf */ + /** 初始化ChunJunCommonConf */ public void initCommonConf(ChunJunCommonConf commonConf) { PropertiesUtil.initCommonConf(commonConf, this.syncConf); commonConf.setCheckFormat(this.syncConf.getWriter().getBooleanVal("check", true)); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/util/PluginUtil.java b/chunjun-core/src/main/java/com/dtstack/chunjun/util/PluginUtil.java index 966232339f..730238567d 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/util/PluginUtil.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/util/PluginUtil.java @@ -144,7 +144,7 @@ public static Set getDdlJarFileDirPath( } /** - * Obtain local and remote FlinkX plugin jar package path + * Obtain local and remote ChunJun plugin jar package path * * @param pluginPath * @param suffix @@ -414,7 +414,7 @@ public static List setPipelineOptionsToEnvConfig( jarList.addAll(urlList); List pipelineJars = new ArrayList(); - LOG.info("Flinkx executionMode: " + executionMode); + LOG.info("ChunJun executionMode: " + executionMode); if (ClusterMode.getByName(executionMode) == ClusterMode.kubernetesApplication) { for (String jarUrl : jarList) { String newJarUrl = jarUrl; @@ -430,7 +430,7 @@ public static List setPipelineOptionsToEnvConfig( pipelineJars.addAll(jarList); } - LOG.info("Flinkx reset pipeline.jars: " + pipelineJars); + LOG.info("ChunJun reset pipeline.jars: " + pipelineJars); configuration.set(PipelineOptions.JARS, pipelineJars); List classpathList = configuration.get(PipelineOptions.CLASSPATHS); diff --git a/chunjun-formats/chunjun-protobuf/pom.xml b/chunjun-formats/chunjun-protobuf/pom.xml index d1cf68cc89..0b99d9677e 100644 --- a/chunjun-formats/chunjun-protobuf/pom.xml +++ b/chunjun-formats/chunjun-protobuf/pom.xml @@ -9,7 +9,7 @@ 4.0.0 - flinkx-protobuf + chunjun-protobuf 8 diff --git a/chunjun-metrics/chunjun-metrics-prometheus/src/main/java/com/dtstack/chunjun/metrics/prometheus/PrometheusReport.java b/chunjun-metrics/chunjun-metrics-prometheus/src/main/java/com/dtstack/chunjun/metrics/prometheus/PrometheusReport.java index 63837f5251..529b55c515 100644 --- a/chunjun-metrics/chunjun-metrics-prometheus/src/main/java/com/dtstack/chunjun/metrics/prometheus/PrometheusReport.java +++ b/chunjun-metrics/chunjun-metrics-prometheus/src/main/java/com/dtstack/chunjun/metrics/prometheus/PrometheusReport.java @@ -141,7 +141,7 @@ public void open() { @Override public void registerMetric(Accumulator accumulator, String name) { - name = Metrics.METRIC_GROUP_KEY_FLINKX + "_" + name; + name = Metrics.METRIC_GROUP_KEY_CHUNJUN + "_" + name; ReporterScopedSettings reporterScopedSettings = new ReporterScopedSettings(0, ',', Collections.emptySet()); FrontMetricGroup front = diff --git a/docs/conectorShare.md b/docs/conectorShare.md deleted file mode 100644 index 9fac0317f1..0000000000 --- a/docs/conectorShare.md +++ /dev/null @@ -1,175 +0,0 @@ -# connector 共用 - -**NOTE:新版chunjun支持connector和flinkSql自带的connector共用。** - -## 在chunjun中使用flinkSql的connector -- **本地调试** - - 在`chunjun-local-test`模块下将对应的flink connector的GAV拷贝到pom.xml文件中 - - 将chunjun中的connector的GAV拷贝到pom.xml文件中(部分flink connector已经通过chunjun connector引入,如果是则上一步省略) - - 在LocalTest类中指定参数运行即可 - sql: - ```sql - -- {"id":100,"name":"lb james阿道夫","money":293.899778,"dateone":"2020-07-30 10:08:22","age":"33","datethree":"2020-07-30 10:08:22.123","datesix":"2020-07-30 10:08:22.123456","datenigth":"2020-07-30 10:08:22.123456789","dtdate":"2020-07-30","dttime":"10:08:22"} - CREATE TABLE source_ods_fact_user_ippv - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'kafka' - ,'topic' = 'da' - ,'properties.bootstrap.servers' = 'localhost:9092' - ,'properties.group.id' = 'luna_g' - ,'scan.startup.mode' = 'earliest-offset' - ,'format' = 'json' - ,'json.timestamp-format.standard' = 'SQL' - ); - - CREATE TABLE result_total_pvuv_min - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'stream-x' - ); - INSERT INTO result_total_pvuv_min - SELECT * - from source_ods_fact_user_ippv; - ``` - pom: - ```text - - org.apache.flink - flink-connector-kafka_2.12 - 1.12.2 - - - flink-core - org.apache.flink - - - - - com.dtstack.chunjun - chunjun-connector-stream - 1.12-SNAPSHOT - - ``` -
- -- **服务器上运行:** - - 1.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到chunjun/lib目录下,构建jobGraph使用 - - 2.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到flink/lib下 - - 3.在在chunjun/lib下目录下执行命令: - - local模式 - ```shell - java -cp "./*" com.dtstack.chunjun.client.Launcher \ - -mode local \ - -jobType sql \ - -jobName flink1.12_SPI \ - -job /yourjobpath/sqlFile.sql \ - -chunjunDistDir /chunjun/chunjun-dist - ``` - ![conectorShare_local.png](images/conectorShare_local.png) - - - yarnPer模式 - ```shell - java -cp "./*" com.dtstack.chunjun.client.Launcher \ - -mode yarnPer \ - -jobType sql \ - -jobName flink1.12_SPI \ - -job /yourjobpath/sqlFile.sql \ - -chunjunDistDir /chunjun/chunjun-dist \ - -flinkConfDir /flink/conf \ - -hadoopConfDir /hadoop/etc \ - -flinkLibDir /flink/lib \ - -queue default - ``` - ![conectorShare_yarnPer.png](images/conectorShare_yarnPer.png) - ![conectorShare_yarn.png](images/conectorShare_yarn.png) -## 在flinkSql中使用chunjun的connector -- **本地调试** - - 在自己项目中将对应的flink connector的GAV拷贝到pom.xml文件中 - - 将chunjun中的core和connector的GAV拷贝到pom.xml文件中(需要先deploy项目) - - 运行自己的任务 - sql: - ```sql - -- {"id":100,"name":"lb james阿道夫","money":293.899778,"dateone":"2020-07-30 10:08:22","age":"33","datethree":"2020-07-30 10:08:22.123","datesix":"2020-07-30 10:08:22.123456","datenigth":"2020-07-30 10:08:22.123456789","dtdate":"2020-07-30","dttime":"10:08:22"} - CREATE TABLE source_ods_fact_user_ippv - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'kafka' - ,'topic' = 'da' - ,'properties.bootstrap.servers' = 'localhost:9092' - ,'properties.group.id' = 'luna_g' - ,'scan.startup.mode' = 'earliest-offset' - ,'format' = 'json' - ,'json.timestamp-format.standard' = 'SQL' - ); - - CREATE TABLE result_total_pvuv_min - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'stream-x' - ); - INSERT INTO result_total_pvuv_min - SELECT * - from source_ods_fact_user_ippv; - ``` - pom: - ```text - - org.apache.flink - flink-sql-connector-kafka_2.12 - 1.12.2 - - - com.dtstack.chunjun - chunjun-core - 1.12-SNAPSHOT - - - com.dtstack.chunjun - chunjun-connector-stream - 1.12-SNAPSHOT - - ``` -
- -- **服务器上运行:** - - 将flink需要的connector和chunjun的chunjun-core-feat_1.12_pluginMerge.jar包和对应connector的jar引入到自己项目的pom中,将项目打成fat包,提交任务即可。 diff --git a/docs/confProp.md b/docs/confProp.md deleted file mode 100644 index c67c161406..0000000000 --- a/docs/confProp.md +++ /dev/null @@ -1,50 +0,0 @@ -## confProp - - * table.exec.source.idle-timeout:当一个源在超时时间内没有收到任何元素时,它将被标记为临时空闲。这允许下游任务推进其水印,而无需在空闲时等待来自该源的水印。默认值为 0(表示未启用检测源空闲),可设置:10 ms(单位毫秒)。 - - - * table.exec.emit.early-fire.enabled:开启window统计提前触发功能。默认:false(表示不开启),设置true开启。 - * table.exec.emit.early-fire.delay:开启window统计提前触发时间,上面设置为true才有效。无默认值,可设置:1s(单位为秒)。 - - - * table.exec.state.ttl:状态最小过期时间。默认:0 ms(代表不过期)。 - - - * table.exec.mini-batch.enabled:是否开启minibatch,可以减少状态开销。这可能会增加一些延迟,因为它会缓冲一些记录而不是立即处理它们。这是吞吐量和延迟之间的权衡。默认:false(表示不开启),设置true开启 - * table.exec.mini-batch.allow-latency:状态缓存时间,table.exec.mini-batch.enabled为true才有效。无默认,可设置:5 s(单位为秒)。 - * table.exec.mini-batch.size:状态最大缓存条数,table.exec.mini-batch.enabled为true才有效。无默认,可设置:5000(单位为条数)。 - * table.optimizer.agg-phase-strategy:是否开启Local-Global聚合,前提需要开启minibatch,聚合是为解决数据倾斜问题提出的,类似于 MapReduce 中的 Combine + Reduce 模式。无默认,可设置:TWO_PHASE。 - - - * table.optimizer.distinct-agg.split.enabled:是否开启拆分distinct聚合,Local-Global可以解决数据倾斜,但是在处理distinct聚合时,其性能并不令人满意,如:SELECT day, COUNT(DISTINCT user_id) FROM T GROUP BY day 如果 distinct key (即 user_id)的值分布稀疏,建议开启。无默认,可设置:true。 - * 其他一些sql相关配置参考:https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/config.html - - - * sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) - * sql.checkpoint.unalignedCheckpoints:是否开启Unaligned Checkpoint,不开启false,开启true。默认为:false。 - * sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] - * sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) - * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 - * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] - - - * state.backend: 任务状态后端,可选为MEMORY,FILESYSTEM,ROCKSDB,默认为flinkconf中的配置。 - * state.checkpoints.dir: FILESYSTEM,ROCKSDB状态后端文件系统存储路径,例如:hdfs://ns1/dtInsight/flink180/checkpoints。 - * state.backend.incremental: ROCKSDB状态后端是否开启增量checkpoint,默认为true。 - * 其他一些state相关配置参考:https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/stream/state/checkpointing.html - - - * sql.env.parallelism: 默认并行度设置 - * sql.max.env.parallelism: 最大并行度设置 - - - * time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] - - - * jobmanager.memory.process.size: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:1600m) - * taskmanager.memory.process.size: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:1728m) - * taskmanager.numberOfTaskSlots:per_job模式下指定每个taskmanager对应的slot数量(默认1),通过该参数和sql.env.parallelism可控制tm的个数,即sql.env.parallelism/taskmanager.numberOfTaskSlots 向上取整。 - * s:任务恢复点的路径(默认无) - * allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false) - * logLevel: 日志级别动态配置(默认info) - * [prometheus 相关参数](./prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 diff --git a/docs/connectorShare.md b/docs/connectorShare.md deleted file mode 100644 index 91beed1ade..0000000000 --- a/docs/connectorShare.md +++ /dev/null @@ -1,150 +0,0 @@ -# connector 共用 - -**NOTE:新版ChunJun支持connector和flinkSql自带的connector共用。** - -## 在ChunJun中使用flinkSql的connector -- **本地调试** - - 在`chunjun-local-test`模块下将对应的flink connector的GAV拷贝到pom.xml文件中 - - 将ChunJun中的connector的GAV拷贝到pom.xml文件中(部分flink connector已经通过chunjun connector引入,如果是则上一步省略) - - 在LocalTest类中指定参数运行即可 - sql: - ```sql - -- {"id":100,"name":"lb james阿道夫","money":293.899778,"dateone":"2020-07-30 10:08:22","age":"33","datethree":"2020-07-30 10:08:22.123","datesix":"2020-07-30 10:08:22.123456","datenigth":"2020-07-30 10:08:22.123456789","dtdate":"2020-07-30","dttime":"10:08:22"} - CREATE TABLE source_ods_fact_user_ippv - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'kafka' - ,'topic' = 'da' - ,'properties.bootstrap.servers' = 'localhost:9092' - ,'properties.group.id' = 'luna_g' - ,'scan.startup.mode' = 'earliest-offset' - ,'format' = 'json' - ,'json.timestamp-format.standard' = 'SQL' - ); - - CREATE TABLE result_total_pvuv_min - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'stream-x' - ); - INSERT INTO result_total_pvuv_min - SELECT * - from source_ods_fact_user_ippv; - ``` - pom: - ```text - - org.apache.flink - flink-connector-kafka_2.12 - 1.12.2 - - - flink-core - org.apache.flink - - - - - com.dtstack.chunjun - chunjun-connector-stream - 1.12-SNAPSHOT - - ``` -
- -- **服务器上运行:** - - 将对应connector的jar放到flinkLib目录下 - - sql任务中建表时,with属性使用原生connector属性即可。 - -## 在flinkSql中使用ChunJun的connector -- **本地调试** - - 在自己项目中将对应的flink connector的GAV拷贝到pom.xml文件中 - - 将ChunJun中的core和connector的GAV拷贝到pom.xml文件中(需要先deploy项目) - - 运行自己的任务 - sql: - ```sql - -- {"id":100,"name":"lb james阿道夫","money":293.899778,"dateone":"2020-07-30 10:08:22","age":"33","datethree":"2020-07-30 10:08:22.123","datesix":"2020-07-30 10:08:22.123456","datenigth":"2020-07-30 10:08:22.123456789","dtdate":"2020-07-30","dttime":"10:08:22"} - CREATE TABLE source_ods_fact_user_ippv - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'kafka' - ,'topic' = 'da' - ,'properties.bootstrap.servers' = 'localhost:9092' - ,'properties.group.id' = 'luna_g' - ,'scan.startup.mode' = 'earliest-offset' - ,'format' = 'json' - ,'json.timestamp-format.standard' = 'SQL' - ); - - CREATE TABLE result_total_pvuv_min - ( - id INT - , name STRING - , money decimal - , dateone timestamp - , age bigint - , datethree timestamp - , datesix timestamp(6) - , datenigth timestamp(9) - , dtdate date - , dttime time - ) WITH ( - 'connector' = 'stream-x' - ); - INSERT INTO result_total_pvuv_min - SELECT * - from source_ods_fact_user_ippv; - ``` - pom: - ```text - - org.apache.flink - flink-sql-connector-kafka_2.12 - 1.12.2 - - - com.dtstack.chunjun - chunjun-core - 1.12-SNAPSHOT - - - com.dtstack.chunjun - chunjun-connector-stream - 1.12-SNAPSHOT - - ``` -
- -- **服务器上运行:** - - 将ChunJun的chunjun-core-feat_1.12_pluginMerge.jar包和对应connector的jar放到flinkLib目录下 - - sql任务中建表时,with属性使用ChunJun connector属性文档中描述的即可。 diff --git a/docs/connectors-en/binlog/binlog-source.md b/docs/connectors-en/binlog/binlog-source.md deleted file mode 100644 index 5feffdd937..0000000000 --- a/docs/connectors-en/binlog/binlog-source.md +++ /dev/null @@ -1,484 +0,0 @@ -# MySQL Binlog Source - - - -- [I. Introduction](# I. Introduction) -- [II. Supported Versions](# II. Supported Versions) -- [III. Plugin Name](# III. Plugin Name) -- [IV. Database Configuration] (# IV. Database Configuration) - - [i. Change Configuration](# i. Change Configuration) - - [ii. Add Permissions](# ii. Add Permissions) -- [V. Parameter Description] (# V. Parameter Description) - - [i. Sync](# i. Sync) - - [ii. Sql](# ii. Sql) -- [VI. Data Structure] (# VI. Data Structure) -- [VII. Data Types](# VII. Data Types) -- [VIII. Script Example](# VIII. Script Example) - - - -## I. Introduction -The MySQL Binlog plugin uses the Canal component to capture the changed data from MySQL in real-time. Currently, the sink-plugin does not support data restoration, and only support to write changed log data. - -## II. Supported Versions -MySQL 5.1.5 and above, TiDB 3.0.10 and later - -## III. Plugin Name -| Sync | binlogsource、binlogreader | -| --- | --- | -| SQL | binlog-x | - -## IV. Database Configuration -### i. Change Configuration -MySQL binlog_format needs to be modified to ROW, add the following configuration under [mysqld] in the /etc/my.cnf file -```sql -server_id=109 -log_bin = /var/lib/mysql/mysql-bin -binlog_format = ROW -expire_logs_days = 30 -``` - -### ii. Add Permissions -MySQL Binlog requires three permissions: SELECT, REPLICATION SLAVE, REPLICATION CLIENT -```sql -GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO'canal'@'%' IDENTIFIED BY'canal'; -``` - - --When the SELECT permission is missing, the error is reported as -```text -com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: -Access denied for user'canal'@'%' to database'binlog' -``` - --When REPLICATION SLAVE permission is missing, the error is reported as -```text -java.io.IOException: -Error When doing Register slave:ErrorPacket [errorNumber=1045, fieldCount=-1, message=Access denied for user'canal'@'%' -``` - --When REPLICATION CLIENT permission is missing, the error is reported as -```text -com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: -Access denied; you need (at least one of) the SUPER, REPLICATION CLIENT privilege(s) for this operation -``` - -Why does Binlog need these permissions: - --Select permission means are allowed to view data onto the table --Replication client permission means are allowed to execute the show master status, show slave status, and show binary logs commands --Replication slave permission means that the slave host is allowed to connect to the master through this user in order to establish a master-slave replication relationship - -## V. Parameter Description -### i. Sync -- **jdbcUrl** - - Description: jdbc url of MySQL database, reference document: [Mysql official document](http://dev.mysql.com/doc/connector- j/en/connector- j- reference- configuration- properties.html) - - Required: Yes - - Field type: string - - Default value: none -
- -- **username** - - Description: The username of the data source - - Required: Yes - - Field type: string - - Default value: none -
- -- **password** - - Description: The password of the username specified by the data source - - Required: Yes - - Field type: string - - Default value: none -
- -- **host** - - Description: The ip of the machine where the MySQL slave is started - - Required: Yes - - Field type: string - - Default value: none -
- -- **port** - - Description: Port to start MySQL slave - - Required: No - - Field type: int - - Default value: 3306 -
- -- **table** - - Description: The data table to be parsed. - - Note: After specifying this parameter, the filter parameter will be invalid, the table and filter are empty, listen to all tables under the schema in jdbcUrl - - Required: No - - Field type: list - - Default value: none -
- -- **filter** - - Description: Perl regular expression for filtering table names - - Note: both table and filter are empty, listen to all tables under the schema in jdbcUrl - - Required: No - - Field type: string - - Default value: none - - example: - - All tables: `.*` or `.*\\..*` - - All tables under canal schema: `canal\\..*` - - Tables starting with canal under canal: `canal\\.canal.*` - - A table under the canal schema: `canal.test1` -
- -- **cat** - - Description: The type of data update that needs to be parsed, including insert, update, and delete - - Note: Fill in the format separated by commas. If it is empty, parse all data update types - - Required: No - - Field type: string - - Default value: none -
- -- **start** - - Description: The starting position of the binlog file to be read - - Note: If it is empty, it will be consumed from the current position, and the priority of timestamp is higher than journal-name+position - - Parameters: - - timestamp: timestamp, the start point of collection is consumed from the specified timestamp; - - journal-name: file name, the start point of collection is consumed from the beginning of the specified file; - - position: the specified position of the file, the start point of collection is consumed from the specified position of the specified file - - Field type: map - - Default value: none -
- -- **pavingData** - - Description: Whether to flatten the parsed json data, see [六、Data structure](#六数据结构) - - Required: No - - Field type: boolean - - Default value: true -
- -- **splitUpdate** - - Description: When the data update type is update, whether to split the update into two data, see [六、Data structure](#六数据结构) - - Required: No - - Field type: boolean - - Default value: false -
- -- **timestampFormat** - - Description: Specify the timestamp format used for input and output, optional values: `SQL`, `ISO_8601` - - Required: No - - Field type: string - - Default value: SQL -
- -- **slaveId** - - Description: The ID of the slave server - - Note: The same MYSQL replication group cannot be duplicated - - Required: No - - Field type: long - - Default value: new Object().hashCode() -
- -- **connectionCharset** - - Description: encoding information - - Required: No - - Field type: string - - Default value: UTF- 8 -
- -- **detectingEnable** - - Description: Whether to turn on the heartbeat - - Required: No - - Field type: boolean - - Default value: true -
- -- **detectingSQL** - - Description: Heartbeat SQL - - Required: No - - Field type: string - - Default value: SELECT CURRENT_DATE -
- - -- **enableTsdb** - - Description: Whether to enable the ability of time series structure - - Required: No - - Field type: boolean - - Default value: true -
- -- **bufferSize** - - Description: Concurrent cache size - - Note: Must be a power of 2 - - Required: No - - Default value: 1024 -
- -- **parallel** - - Description: Whether to enable parallel parsing of binlog logs - - Required: No - - Field type: boolean - - Default value: true -
- -- **parallelThreadSize** - - Description: Parallel parsing of binlog log threads - - Note: Only when paraller is set to true will it take effect - - Required: No - - Field type: int - - Default value: 2 -
- -- **isGTIDMode** - - Description: Whether to enable gtid mode - - Required: No - - Field type: boolean - - Default value: false -
- -- **queryTimeOut** - - Description: After sending data through the TCP connection (here is the SQL to be executed), the timeout period for waiting for a response, in milliseconds - - Required: No - - Field type: int - - Default value: 300000 -
- -- **connectTimeOut** - - Description: The timeout period for the database driver (mysql-connector-java) to establish a TCP connection with the mysql server, in milliseconds - - Required: No - - Field type: int - - Default value: 60000 -
- -### vii. SQL -- **url** - - Description: jdbc url of MySQL database, reference document: [Mysql official document](http://dev.mysql.com/doc/connector- j/en/connector- j- reference- configuration- properties.html) - - Required: Yes - - Field type: string - - Default value: none -
- -- **username** - - Description: The username of the data source - - Required: Yes - - Field type: string - - Default value: none -
- -- **password** - - Description: The password of the username specified by the data source - - Required: Yes - - Field type: string - - Default value: none -
- -- **host** - - Description: The ip of the machine where the MySQL slave is started - - Required: Yes - - Field type: string - - Default value: none -
- -- **port** - - Description: Port to start MySQL slave - - Required: No - - Field type: int - - Default value: 3306 -
- -- **table** - - Description: The data table to be parsed. - - Note: After specifying this parameter, the filter parameter will be invalid, and the SQL task only supports monitoring a single table - - Required: No - - Field type: string - - Default value: none -
- -- **filter** - - Description: Perl regular expression for filtering table names - - Note: The SQL task only supports monitoring a single table - - Required: No - - Field type: string - - Default value: none - - Example: a table under the canal schema: `canal.test1` -
- -- **cat** - - Description: The type of data update that needs to be parsed, including insert, update, and delete - - Note: Fill in the format separated by commas. If it is empty, parse all data update types - - Required: No - - Field type: string - - Default value: none -
- -- **timestamp** - - Description: The starting position and timestamp of the binlog file to be read, the start point of collection is consumed from the specified timestamp; - - Required: No - - Field type: string - - Default value: none -
- -- **journal- name** - - Description: The start position of the binlog file to be read, the file name, and the start point of collection is consumed from the start of the specified file; - - Required: No - - Field type: string - - Default value: none -
- -- **position** - - Description: The start position of the binlog file to be read, the specified position of the file, the start point of collection is consumed from the specified position of the specified file - - Required: No - - Field type: string - - Default value: none -
- -- **connection- charset** - - Description: encoding information - - Required: No - - Field type: string - - Default value: UTF- 8 -
- -- **detecting- enable** - - Description: Whether to turn on the heartbeat - - Required: No - - Field type: boolean - - Default value: true -
- -- **detecting- sql** - - Description: Heartbeat SQL - - Required: No - - Field type: string - - Default value: SELECT CURRENT_DATE -
- -- **enable- tsdb** - - Description: Whether to enable the ability of time series structure - - Required: No - - Field type: boolean - - Default value: true -
- -- **buffer- size** - - Description: Concurrent cache size -- Note: Must be a power of 2 - - Required: No - - Default value: 1024 -
- -- **parallel** - - Description: Whether to enable parallel parsing of binlog logs - - Required: No - - Field type: boolean - - Default value: true -
- -- **parallel- thread- size** - - Description: Parallel parsing of binlog log threads - - Note: Only when parallel is set to true will it take effect - - Required: No - - Field type: int - - Default value: 2 -
- -- **is- gtid- mode** - - Description: Whether to enable gtid mode - - Required: No - - Field type: boolean - - Default value: false -
- -- **query-time-out** - - Description: After sending data through the TCP connection (here is the SQL to be executed), the timeout period for waiting for a response, in milliseconds - - Required: No - - Field type: int - - Default value: 300000 -
- -- **connect-time-out** - - Description: The timeout period for the database driver (mysql-connector-java) to establish a TCP connection with the mysql server, in milliseconds - - Required: No - - Field type: int - - Default value: 60000 -
- -- **timestamp- format.standard** - - Description: Same as the `timestampFormat` parameter in Sync, specify the timestamp format used for input and output, optional values: `SQL`, `ISO_8601` - - Required: No - - Field type: string - - Default value: SQL - -## VI. Data Structure -Execute at 2020-01-01 12:30:00 (time stamp: 1577853000000): -```sql -INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) VALUES (1, 1,'a'); -``` -Execute at 2020-01-01 12:31:00 (time stamp: 1577853060000): -```sql -DELETE FROM `tudou`.`kudu` WHERE `id` = 1 AND `user_id` = 1 AND `name` ='a'; -``` -Execute at 2020-01-01 12:32:00 (time stamp: 1577853180000): -```sql -UPDATE `tudou`.`kudu` SET `id` = 2, `user_id` = 2, `name` ='b' WHERE `id` = 1 AND `user_id` = 1 AND `name` ='a'; -``` -1. pavingData = true, splitUpdate = false - The data in RowData are as follows: -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", 1, 1, "a", 2, 2, "b"] -``` -2. pavingData = false, splitUpdate = false - The data in RowData are as follows: -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", {"id":1, "user_id":1, "name":"a"}, {"id":2, "user_id": 2, "name":"b"}] -``` -3. pavingData = true, splitUpdate = true - The data in RowData are as follows: -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] - -//schema, table, ts, opTime, type, before_id, before_user_id, before_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", 1, 1, "a"] - -//schema, table, ts, opTime, type, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", 2, 2, "b"] -``` -4. pavingData = false, splitUpdate = true - The data in RowData are as follows: -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] -//schema, table, ts, opTime, type, before -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", {"id":1, "user_id":1, "name":"a"}] -//schema, table, ts, opTime, type, after -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", {"id":2, "user_id":2, "name":"b"}] -``` - --type: change type, INSERT, UPDATE, DELETE --opTime: the execution time of SQL in the database --ts: Self-incrementing ID, not repeated, can be used for sorting, after decoding, it is the event time of ChunJun, the decoding rules are as follows: -```java -long id = Long.parseLong("6760525407742726144"); -long res = id >> 22; -DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -System.out.println(sdf.format(res)); //2021-01-28 19:54:21 -``` - -## VII. Data Types -| Support | BIT | -| --- | --- | -| | TINYINT, SMALLINT, MEDIUMINT, INT, INT24, INTEGER, FLOAT, DOUBLE, REAL, LONG, BIGINT, DECIMAL, NUMERIC | -| | CHAR, VARCHAR, TINYTEXT, TEXT, MEDIUMTEXT, LONGTEXT, ENUM, SET, JSON | -| | DATE, TIME, TIMESTAMP, DATETIME, YEAR | -| | TINYBLOB, BLOB, MEDIUMBLOB, LONGBLOB, GEOMETRY, BINARY, VARBINARY | -| Not currently supported | None | - - -## VIII. Script Example -See the `chunjun-examples` folder in the project. diff --git a/docs/connectors-en/elasticsearch5/es5-sink.md b/docs/connectors-en/elasticsearch5/es5-sink.md deleted file mode 100644 index 9819a7d205..0000000000 --- a/docs/connectors-en/elasticsearch5/es5-sink.md +++ /dev/null @@ -1,119 +0,0 @@ -# 一、Introduce - -The ElasticSearch Sink plugin supports writing data to the specified index. -​ - -# 二、Version support - -Elasticsearch 5.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | elasticsearch5writer | -| SQL | elasticsearch5-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- type - - Description:Elasticsearch document type. - - Required:required - - Type:String - - Default:none -- cluster - - Description:name of connected es's cluster. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- keyDelimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: - -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- sink.bulk-flush.max-actions - - Description:Maximum number of buffered actions per bulk request. Can be set to '0' to disable it. - - Required:optional - - Type:Integer - - Default:1 -- document-id.key-delimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" - -# 五、Data type - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch5/es5-source.md b/docs/connectors-en/elasticsearch5/es5-source.md deleted file mode 100644 index beca31126a..0000000000 --- a/docs/connectors-en/elasticsearch5/es5-source.md +++ /dev/null @@ -1,112 +0,0 @@ -# 一、Introduce - -The ElasticSearch Source plugin supports reading data from an existing ElasticSearch cluster in a specified index. - -# 二、Version support - -Elasticsearch 5.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | elasticsearch5reader | -| SQL | elasticsearch5-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- type - - Description:Elasticsearch document type. - - Required:required - - Type:String - - Default:none -- cluster - - Description:name of connected es's cluster. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: - -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- bulk-flush.max-actions - - Description:Maximum number of actions to buffer for each bulk request. - - Required:否 - - Type:Integer - - Default:1000 - -​ - -# 五、Data type - - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch6/es6-lookup.md b/docs/connectors-en/elasticsearch6/es6-lookup.md deleted file mode 100644 index 91703a5245..0000000000 --- a/docs/connectors-en/elasticsearch6/es6-lookup.md +++ /dev/null @@ -1,94 +0,0 @@ -# 一、介绍 - -The ElasticSearch Lookup plug-in reads data from an existing ElasticSearch cluster in the specified index and associates it with the master table as a dimension table. -Currently, the full dimension table and asynchronous dimension table are supported. - -# 二、Version support - -Elasticsearch 6.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| SQL | es-x | - - -​
- -# 四、Param description - -## 1、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- type - - Description:Elasticsearch document type. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- lookup.cache-type - - Description:Dimension table type. Eg: ALL or LRU - - Required:optional - - Type:String - - Default:LRU -- lookup.cache-period - - Description:Full dimension table period time - - Required:optional - - Type:Long - - Default:3600 * 1000L -- lookup.cache.max-rows - - Description:Maximum number of entries in the dimension table cache. - - Required:optional - - Type:Long - - Default:1000L -- lookup.cache.ttl - - Description:Time To Live. - - Required:optional - - Type:Long - - Default:60 * 1000L -- lookup.error-limit - - Description:Number of non-compliant data in the dimension table. - - Required:optional - - Type:Long - - Default:Long.MAX_VALUE -- lookup.fetch-size - - Description:Fetch the number of items of dimension table data. - - Required:optional - - Type:Integer - - Default:1000L -- lookup.parallelism - - Description:Dimension table parallelism. - - Required:optional - - Type:Integer - - Default:1 - - -# 五、Data type - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch6/es6-sink.md b/docs/connectors-en/elasticsearch6/es6-sink.md deleted file mode 100644 index d80a4a8329..0000000000 --- a/docs/connectors-en/elasticsearch6/es6-sink.md +++ /dev/null @@ -1,114 +0,0 @@ -# 一、Introduce - -The ElasticSearch Sink plugin supports writing data to the specified index. -​ - -# 二、Version support - -Elasticsearch 6.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | eswriter、essink | -| SQL | es-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- type - - Description:Elasticsearch document type. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- keyDelimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: - -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- sink.bulk-flush.max-actions - - Description:Maximum number of buffered actions per bulk request. Can be set to '0' to disable it. - - Required:optional - - Type:Integer - - Default:1 -- document-id.key-delimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" - -# 五、Data type - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch6/es6-source.md b/docs/connectors-en/elasticsearch6/es6-source.md deleted file mode 100644 index fed956d861..0000000000 --- a/docs/connectors-en/elasticsearch6/es6-source.md +++ /dev/null @@ -1,107 +0,0 @@ -# 一、Introduce - -The ElasticSearch Source plugin supports reading data from an existing ElasticSearch cluster in a specified index. - -# 二、Version support - -Elasticsearch 6.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | esreader、essource | -| SQL | es-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: - -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- type - - Description:Elasticsearch document type. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- bulk-flush.max-actions - - Description:Maximum number of actions to buffer for each bulk request. - - Required:否 - - Type:Integer - - Default:1000 - -​ - -# 五、Data type - - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch7/es7-lookup.md b/docs/connectors-en/elasticsearch7/es7-lookup.md deleted file mode 100644 index 7b330b659e..0000000000 --- a/docs/connectors-en/elasticsearch7/es7-lookup.md +++ /dev/null @@ -1,113 +0,0 @@ -# 一、介绍 - -The ElasticSearch Lookup plug-in reads data from an existing ElasticSearch cluster in the specified index and associates it with the master table as a dimension table. -Currently, the full dimension table and asynchronous dimension table are supported. - -# 二、Version support - -Elasticsearch 7.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| SQL | elasticsearch7-x | - - -​
- -# 四、Param description - -## 1、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: "localhost:9200", Multiple addresses are delimited by semicolons. - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- client.connect-timeout - - Description:Elasticsearch client max connect timeout. - - Required:optional - - Type:Integer - - Default:5000 -- client.socket-timeout - - Description:Elasticsearch client max socket timeout. - - Required:optional - - Type:Integer - - Default:1800000 -- client.keep-alive-time - - Description:Elasticsearch client connection max keepAlive time. - - Required:optional - - Type:Integer - - Default:5000 -- client.request-timeout - - Description:Elasticsearch client connection max request timeout. - - Required:optional - - Type:Integer - - Default:2000 -- client.max-connection-per-route - - Description:Elasticsearch client connection assigns maximum connection per route value. - - Required:optional - - Type:Integer - - Default:10 -- lookup.cache-type - - Description:Dimension table type. Eg: ALL or LRU - - Required:optional - - Type:String - - Default:LRU -- lookup.cache-period - - Description:Full dimension table period time - - Required:optional - - Type:Long - - Default:3600 * 1000L -- lookup.cache.max-rows - - Description:Maximum number of entries in the dimension table cache. - - Required:optional - - Type:Long - - Default:1000L -- lookup.cache.ttl - - Description:Time To Live. - - Required:optional - - Type:Long - - Default:60 * 1000L -- lookup.error-limit - - Description:Number of non-compliant data in the dimension table. - - Required:optional - - Type:Long - - Default:Long.MAX_VALUE -- lookup.fetch-size - - Description:Fetch the number of items of dimension table data. - - Required:optional - - Type:Integer - - Default:1000L -- lookup.parallelism - - Description:Dimension table parallelism. - - Required:optional - - Type:Integer - - Default:1 - -# 五、Data type - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch7/es7-sink.md b/docs/connectors-en/elasticsearch7/es7-sink.md deleted file mode 100644 index 2e8b69f5a8..0000000000 --- a/docs/connectors-en/elasticsearch7/es7-sink.md +++ /dev/null @@ -1,158 +0,0 @@ -# 一、Introduce - -The ElasticSearch Sink plugin supports writing data to the specified index. -​ - -# 二、Version support - -Elasticsearch 7.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | elasticsearch7writer | -| SQL | elasticsearch7-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- keyDelimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: -- connectTimeout - - Description:Elasticsearch client max connect timeout. - - Required:optional - - Type:Integer - - Default:5000 -- socketTimeout - - Description:Elasticsearch client max socket timeout. - - Required:optional - - Type:Integer - - Default:1800000 -- keepAliveTime - - Description:Elasticsearch client connection max keepAlive time. - - Required:optional - - Type:Integer - - Default:5000 -- requestTimeout - - Description:Elasticsearch client connection max request timeout. - - Required:optional - - Type:Integer - - Default:2000 -- maxConnPerRoute - - Description:Elasticsearch client connection assigns maximum connection per route value. - - Required:optional - - Type:Integer - - Default:10 -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: "localhost:9200", Multiple addresses are delimited by semicolons. - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- sink.bulk-flush.max-actions - - Description:Maximum number of buffered actions per bulk request. Can be set to '0' to disable it. - - Required:optional - - Type:Integer - - Default:1 -- document-id.key-delimiter - - Description:Delimiter for composite keys ("_" by default), eg:“${col1}_${col2}” - - Required:optional - - Type:String - - Default:"_" -- client.connect-timeout - - Description:Elasticsearch client max connect timeout. - - Required:optional - - Type:Integer - - Default:5000 -- client.socket-timeout - - Description:Elasticsearch client max socket timeout. - - Required:optional - - Type:Integer - - Default:1800000 -- client.keep-alive-time - - Description:Elasticsearch client connection max keepAlive time. - - Required:optional - - Type:Integer - - Default:5000 -- client.request-timeout - - Description:Elasticsearch client connection max request timeout. - - Required:optional - - Type:Integer - - Default:2000 -- client.max-connection-per-route - - Description:Elasticsearch client connection assigns maximum connection per route value. - - Required:optional - - Type:Integer - - Default:10 - -# 五、Data type - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/elasticsearch7/es7-source.md b/docs/connectors-en/elasticsearch7/es7-source.md deleted file mode 100644 index 59ec65a8e2..0000000000 --- a/docs/connectors-en/elasticsearch7/es7-source.md +++ /dev/null @@ -1,151 +0,0 @@ -# 一、Introduce - -The ElasticSearch Source plugin supports reading data from an existing ElasticSearch cluster in a specified index. - -# 二、Version support - -Elasticsearch 7.x -​ -# 三、Plugin name - -| type|name| -| ---- | ----| -| Sync | elasticsearch7reader | -| SQL | elasticsearch7-x | - -​ - -# 四、Param description - -## 1、Sync - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: ["localhost:9200"] - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:none -- batchSize - - Description:Number of data pieces written in batches - - Required:optional - - Type:Integer - - Default:1 -- column - - Description:Columns to be synchronized - - note:'*' is not supported. - - format: -- connectTimeout - - Description:Elasticsearch client max connect timeout. - - Required:optional - - Type:Integer - - Default:5000 -- socketTimeout - - Description:Elasticsearch client max socket timeout. - - Required:optional - - Type:Integer - - Default:1800000 -- keepAliveTime - - Description:Elasticsearch client connection max keepAlive time. - - Required:optional - - Type:Integer - - Default:5000 -- requestTimeout - - Description:Elasticsearch client connection max request timeout. - - Required:optional - - Type:Integer - - Default:2000 -- maxConnPerRoute - - Description:Elasticsearch client connection assigns maximum connection per route value. - - Required:optional - - Type:Integer - - Default:10 -``` -"column": [{ - "name": "col", -- Column name, which can be found in a multi-level format - "type": "string", -- Column type, when name is not specified, returns a constant column with the value specified by value - "value": "value" -- Constant column value -}] -``` -​ - -## 2、SQL - -- hosts - - Description:One or more Elasticsearch hosts to connect to。eg: "localhost:9200", Multiple addresses are delimited by semicolons. - - Required:required - - Type:List - - Default:none -- index - - Description:Elasticsearch index for every record. - - Required:required - - Type:String - - Default:none -- username - - Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster. - - Required:optional - - Type:String - - Default:none -- password - - Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. - - Required:optional - - Type:String - - Default:无 -- bulk-flush.max-actions - - Description:Maximum number of actions to buffer for each bulk request. - - Required:否 - - Type:Integer - - Default:1000 -- client.connect-timeout - - Description:Elasticsearch client max connect timeout. - - Required:optional - - Type:Integer - - Default:5000 -- client.socket-timeout - - Description:Elasticsearch client max socket timeout. - - Required:optional - - Type:Integer - - Default:1800000 -- client.keep-alive-time - - Description:Elasticsearch client connection max keepAlive time. - - Required:optional - - Type:Integer - - Default:5000 -- client.request-timeout - - Description:Elasticsearch client connection max request timeout. - - Required:optional - - Type:Integer - - Default:2000 -- client.max-connection-per-route - - Description:Elasticsearch client connection assigns maximum connection per route value. - - Required:optional - - Type:Integer - - Default:10 - -​ - -# 五、Data type - - -|supported | date type | -| --- | --- | -| yes |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| no | IP,binary, nested, object| - - -# 六、Sample demo - -See the 'demo' folder in the 'ChunJun: Local: Test' module of the project. diff --git a/docs/connectors-en/hdfs/hdfs-sink-en.md b/docs/connectors-en/hdfs/hdfs-sink-en.md deleted file mode 100644 index 8c90b1c473..0000000000 --- a/docs/connectors-en/hdfs/hdfs-sink-en.md +++ /dev/null @@ -1,320 +0,0 @@ -# HDFS Sink - -## Ⅰ、Introduction -The HDFS plugin supports reading and writing TextFile, Orc, and Parquet files directly from the configured HDFS path, and is generally used with Hive tables. For example: reading all data in a partition of the Hive table is essentially reading the data files under the HDFS path of the corresponding partition of the Hive table; writing data to a partition of the Hive table is essentially writing the data files directly to the HDFS of the corresponding partition Under the path; the HDFS plugin will not perform any DDL operations on the Hive table. -HDFS Sink will use two-phase commit when checkpoint is turned on. During pre-commit, the data files generated in the .data directory are copied to the official directory and the copied data files are marked. The data files marked in the .data directory are deleted during the commit phase and rolled back. Delete the data files marked in the official catalog at the time. - - -## Ⅱ、Supported version -Hadoop 2.x、Hadoop 3.x - - -## Ⅲ、Plugin name -| Sync | hdfssink、hdfswriter | -| --- | --- | -| SQL | hdfs-x | - - -## Ⅳ、Parameter Description - -### 1、Sync -- **path** - - description:The path of the data file to be written - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **fileName** - - description:Data file directory name - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **writeMode** - - description:HDFS Sink data cleaning processing mode before writing: - - append - - overwrite - - notice:All files in the current directory of hdfs will be deleted in overwrite mode - - required:required - - type:string - - defaults:append -
- -- **fileType** - - description:○ File type, currently only supports user configuration as `text`, `orc`, `parquet` - - text:textfile file format - - orc:orcfile file format - - parquet:parquet file format - - required:required - - type:string - - defaults:none -
- -- **defaultFS** - - description:Hadoop hdfs file system namenode node address. Format: hdfs://ip:port; for example: hdfs://127.0.0.1:9000 - - required:required - - type:string - - defaults:none -
- -- **column** - - description:Need to read the field - - notice:Does not support * format - - format: -```text -"column": [{ - "name": "col", - "type": "string", - "index":1, - "isPart":false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] -``` - -- property description: - - name:required,Field Name - - type:required,Field type, which needs to match the actual field type in the data file - - index:optional,The position index of the field in all fields, starting from 0, the default is -1, read in sequence in the order of the array, read the specified field column after configuration - - isPart:optional,Whether it is a partition field, if it is a partition field, the partition assignment will be automatically intercepted from the path, the default is fale - - format:optional,Format the date according to the specified format - - value:optional,Constant field, return the value of value as a constant column -- required:required -- type:Array -- defaults:none -
- -- **hadoopConfig** - - description:The configuration in core-site.xml and hdfs-site.xml that need to be filled in the cluster HA mode, including kerberos related configuration when kerberos is turned on - - required:optional - - type:Map - - defaults:none -
- -- **fieldDelimiter** - - description: Field separator when fileType is text - - required:optional - - type:string - - defaults:`\001` -
- -- **fullColumnName** - - description:Field name written - - required:optional - - type:list - - defaults:column name collection -
- -- **fullColumnType** - - description:Field type written - - required:optional - - type:list - - defaults:column type collection -
- -- **compress** - - description:hdfs file compression type - - text:Support `GZIP`, `BZIP2` format - - orc:Support`SNAPPY`、`GZIP`、`BZIP`、`LZ4`format - - parquet:Support`SNAPPY`、`GZIP`、`LZO`format - - notice:`SNAPPY`format requires users to install SnappyCodec - - required:optional - - type:string - - defaults: - - text is not compressed by default - - orc defaults to ZLIB format - - parquet defaults to SNAPPY format -
- -- **maxFileSize** - - description:The maximum size of a single file written to hdfs, in bytes - - required:optional - - type:long - - defaults:`1073741824`(1G) -
- -- **nextCheckRows** - - description:The number of intervals for checking the file size next time, and the file size of the current written file will be queried every time this number is reached - - required:optional - - type:long - - defaults:`5000` -
- -- **rowGroupSIze** - - description:Set the size of the row group when the fileType is parquet, in bytes - - required:optional - - type:int - - defaults:`134217728`(128M) -
- -- **enableDictionary** - - description:When fileType is parquet, whether to start dictionary encoding - - required:optional - - type:boolean - - defaults:`true` -
- -- **encoding** - - description:The character encoding of the field when fileType is text - - required:optional - - type:string - - defaults:`UTF-8` - - -### 2、SQL -- **path** - - description:The path of the data file to be written - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **file-name** - - description:Data file directory name - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **write-mode** - - description:HDFS Sink data cleaning processing mode before writing: - - append - - overwrite - - notice:All files in the current directory of hdfs will be deleted in overwrite mode - - required:required - - type:string - - defaults:append -
- -- **file-type** - - description:○ File type, currently only supports user configuration as `text`, `orc`, `parquet` - - text:textfile file format - - orc:orcfile file format - - parquet:parquet file format - - required:required - - type:string - - defaults:none -
- -- **default-fs** - - description:Hadoop hdfs file system namenode node address. Format: hdfs://ip:port; for example: hdfs://127.0.0.1:9000 - - required:required - - type:string - - defaults:none -
- -- **column** - - description:Need to read the field - - notice:Does not support * format - - format: -```text -"column": [{ - "name": "col", - "type": "string", - "index":1, - "isPart":false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] -``` - -- property description: - - name:required,Field Name - - type:required,Field type, which needs to match the actual field type in the data file - - index:optional,The position index of the field in all fields, starting from 0, the default is -1, read in sequence in the order of the array, read the specified field column after configuration - - isPart:optional,Whether it is a partition field, if it is a partition field, the partition assignment will be automatically intercepted from the path, the default is fale - - format:optional,Format the date according to the specified format - - value:optional,Constant field, return the value of value as a constant column -- required:required -- type:Array -- defaults:none -
- -- **hadoopConfig** - - description:The configuration in core-site.xml and hdfs-site.xml that need to be filled in the cluster HA mode, including kerberos related configuration when kerberos is turned on - - required:optional - - defaults:none - - configuration method:'properties.key' ='value', key is the key in hadoopConfig, and value is the value in hadoopConfig, as shown below: -``` -'properties.hadoop.user.name' = 'root', -'properties.dfs.ha.namenodes.ns' = 'nn1,nn2', -'properties.fs.defaultFS' = 'hdfs://ns', -'properties.dfs.namenode.rpc-address.ns.nn2' = 'ip:9000', -'properties.dfs.client.failover.proxy.provider.ns' = 'org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider', -'properties.dfs.namenode.rpc-address.ns.nn1' = 'ip:9000', -'properties.dfs.nameservices' = 'ns', -'properties.fs.hdfs.impl.disable.cache' = 'true', -'properties.fs.hdfs.impl' = 'org.apache.hadoop.hdfs.DistributedFileSystem' -``` - -- **field-delimiter** - - description: Field separator when fileType is text - - required:optional - - type:string - - defaults:`\001` -
- -- **compress** - - description:hdfs file compression type - - text:Support `GZIP`, `BZIP2` format - - orc:Support`SNAPPY`、`GZIP`、`BZIP`、`LZ4`format - - parquet:Support`SNAPPY`、`GZIP`、`LZO`format - - notice:`SNAPPY`format requires users to install SnappyCodec - - required:optional - - type:string - - defaults: - - text is not compressed by default - - orc defaults to ZLIB format - - parquet defaults to SNAPPY format -
- -- **max-file-size** - - description:The maximum size of a single file written to hdfs, in bytes - - required:optional - - type:long - - defaults:`1073741824`(1G) -
- -- **next-check-rows** - - description:The number of intervals for checking the file size next time, and the file size of the current written file will be queried every time this number is reached - - required:optional - - type:long - - defaults:`5000` -
- -- **enable-dictionary** - - description:When fileType is parquet, whether to start dictionary encoding - - required:optional - - type:boolean - - defaults:`true` -
- -- **encoding** - - description:The character encoding of the field when fileType is text - - required:optional - - type:string - - defaults:`UTF-8` - -- **sink.parallelism** - - description:sink parallelism - - required:optional - - type:string - - defaults:none -
- - -## Ⅴ、Data Type -| Support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| Not supported yet | ARRAY、MAP、STRUCT、UNION | - - -## Ⅵ、Script example -See the `chunjun-examples` folder in the project. - diff --git a/docs/connectors-en/hdfs/hdfs-source-en.md b/docs/connectors-en/hdfs/hdfs-source-en.md deleted file mode 100644 index 80f78a3edd..0000000000 --- a/docs/connectors-en/hdfs/hdfs-source-en.md +++ /dev/null @@ -1,199 +0,0 @@ -# HDFS Source - -## Ⅰ、Introduction -The HDFS plugin supports reading and writing TextFile, Orc, and Parquet files directly from the configured HDFS path, and is generally used with Hive tables. For example: reading all data in a partition of the Hive table is essentially reading the data files under the HDFS path of the corresponding partition of the Hive table; writing data to a partition of the Hive table is essentially writing the data files directly to the HDFS of the corresponding partition Under the path; the HDFS plugin will not perform any DDL operations on the Hive table. -HDFS Source does not save the offset of the read file during checkpoint, so it does not support continued running. - -## Ⅱ、Supported version -Hadoop 2.x、Hadoop 3.x - - -## Ⅲ、Plugin name -| Sync | hdfssource、hdfsreader | -| --- | --- | -| SQL | hdfs-x | - - -## Ⅳ、Parameter Description -### 1、Sync -- **path** - - description:The path of the data file to be written - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **fileName** - - description:Data file directory name - - notice:The file path actually written is path/fileName - - required:required - - type:string - - defaults:none -
- -- **defaultFS** - - description:Hadoop hdfs file system namenode node address. Format: hdfs://ip:port; for example: hdfs://127.0.0.1:9000 - - required:required - - type:String - - defaults:none -
- -- **column** - - description:Need to read the field - - notice:Does not support * format - - format: - ```text - "column": [{ - "name": "col", - "type": "string", - "index": 1, - "isPart": false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - property description: - - name:required,Field Name - - type:required,Field type, which needs to match the actual field type in the data file - - index:optional,The position index of the field in all fields, starting from 0, the default is -1, read in sequence in the order of the array, read the specified field column after configuration - - isPart:optional,Whether it is a partition field, if it is a partition field, the partition assignment will be automatically intercepted from the path, the default is fale - - format:optional,Format the date according to the specified format - - value:optional,Constant field, return the value of value as a constant column - - required:required - - type:Array - - defaults:none -
- -- **hadoopConfig** - - description:The configuration in core-site.xml and hdfs-site.xml that need to be filled in the cluster HA mode, including kerberos related configuration when kerberos is turned on - - required:optional - - type:Map - - defaults:none -
- -- **filterRegex** - - description:File regular expression, read the matched file - - required:optional - - type:String - - defaults:none -
- -- **fieldDelimiter** - - description: Field separator when fileType is text - - required:optional - - type:String - - defaults:`\001` -
- -- **encoding** - - description:The character encoding of the field when fileType is text - - required:optional - - type:String - - defaults:`UTF-8` - - -### 2、SQL -- **path** - - description:The path of the data file to be read - - required:required - - type:String - - defaults:none -
- -- **file-type** - - description:○ File type, currently only supports user configuration as `text`, `orc`, `parquet` - - text:textfile file format - - orc:orcfile file format - - parquet:parquet file format - - required:required - - type:String - - defaults:none -
- -- **default-fs** - - description:Hadoop hdfs file system namenode node address. Format: hdfs://ip:port; for example: hdfs://127.0.0.1:9000 - - required:required - - type:String - - defaults:none -
- -- **column** - - description:Need to read the field - - notice:Does not support * format - - format: - ```text - "column": [{ - "name": "col", - "type": "string", - "index": 1, - "isPart": false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - property description: - - name:required,Field Name - - type:required,Field type, which needs to match the actual field type in the data file - - index:optional,The position index of the field in all fields, starting from 0, the default is -1, read in sequence in the order of the array, read the specified field column after configuration - - isPart:optional,Whether it is a partition field, if it is a partition field, the partition assignment will be automatically intercepted from the path, the default is fale - - format:optional,Format the date according to the specified format - - value:optional,Constant field, return the value of value as a constant column - - required:required - - type:Array - - defaults:none -
- -- **hadoopConfig** - - description:The configuration in core-site.xml and hdfs-site.xml that need to be filled in the cluster HA mode, including kerberos related configuration when kerberos is turned on - - required:optional - - defaults:none - - configuration method:'properties.key' ='value', key is the key in hadoopConfig, and value is the value in hadoopConfig, as shown below: - ```text - 'properties.hadoop.user.name' = 'root', - 'properties.dfs.ha.namenodes.ns' = 'nn1,nn2', - 'properties.fs.defaultFS' = 'hdfs://ns', - 'properties.dfs.namenode.rpc-address.ns.nn2' = 'ip:9000', - 'properties.dfs.client.failover.proxy.provider.ns' = 'org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider', - 'properties.dfs.namenode.rpc-address.ns.nn1' = 'ip:9000', - 'properties.dfs.nameservices' = 'ns', - 'properties.fs.hdfs.impl.disable.cache' = 'true', - 'properties.fs.hdfs.impl' = 'org.apache.hadoop.hdfs.DistributedFileSystem' - ``` - -- **filter-regex** - - description:File regular expression, read the matched file - - required:optional - - type:String - - defaults:none -
- -- **field-delimiter** - - description: Field separator when fileType is text - - required:optional - - type:String - - defaults:`\001` -
- -- **encoding** - - description:The character encoding of the field when fileType is text - - required:optional - - type:String - - defaults:`UTF-8` - -- **scan.parallelism** - - description:source parallelism - - required:optional - - type:String - - defaults:none -
- - -## Ⅴ、Data Type -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| Not supported yet | ARRAY、MAP、STRUCT、UNION | - - -## Ⅵ、Script example -See the `chunjun-examples` folder in the project. diff --git a/docs/connectors-en/kafka/kafka-sink.md b/docs/connectors-en/kafka/kafka-sink.md deleted file mode 100644 index 3ad83ee11b..0000000000 --- a/docs/connectors-en/kafka/kafka-sink.md +++ /dev/null @@ -1,172 +0,0 @@ -# Kafka Sink - -## 1. Introduce - -kafka sink - -## 2. Version Support -Kafka mainstream version - -## 3. Connector Name - -| Sync | kafkasink、kafkawriter | -| --- | --- | -| SQL | kafka-x | - -## 4. Parameter description - -### 4.1. Sync - -- **topic** - - Description:Topic name of the Kafka record. - - Required:required - - Type:String - - Default:(none) -
- -- **consumerSettings** - - Description:This can set and pass arbitrary Kafka configurations. It supports all options in `kafka.consumer.ConsumerConfig.ConsumerConfig` class. - - Required:required - - Type:Map - - Default:(none) - - Example: - ```json - { - "consumerSettings":{ - "bootstrap.servers":"host1:9092,host2:9092,host3:9092" - } - } - ``` - -
- -- **tableFields** - - Description:reader and writer fields mapping。If this option is set, the key of JSON will subject to it. - - Note: - - If this option is set, the number of writer column can't less than reader. Otherwise, the configuration will be disabled. - - The mapping relationship is matched according to the sequence of fields. -- Required:optional - - Type:String[] - - Default:(none) -
- -### 4.2. SQL - -The details are in [kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) - -- **connector** - - Description:kafka-x - - Required:required - - Type:String - - Default:(none) -
- -- **topic** - - Description:Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified for sources. When the table is used as sink, the topic name is the topic to write data to. Note topic list is not supported for sinks. - - Required:required - - Type:String - - Default:(none) -
- -- **topic-pattern** - - Description:The regular expression for a pattern of topic names to read from. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. Note, only one of "topic-pattern" and "topic" can be specified for sources. - - Required:optional - - Type:String - - Default:(none) -
- -- **properties.bootstrap.servers** - - Description:Comma separated list of Kafka brokers. - - Required:required - - Type:String - - Default:(none) -
- -- **properties.group.id** - - Description:The id of the consumer group for Kafka source, optional for Kafka sink. - - Required:required by source - - Type:String - - Default:(none) -
- -- **properties.*** - - Description:This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in [Kafka Configuration documentation](https://kafka.apache.org/documentation/#configuration). Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. - - Required:optional - - Type:String - - Default:(none) -
- -- **format** - - Description: The format used to deserialize and serialize the value part of Kafka messages. Please refer to the [format](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) page for more details and more format options. Note: Either this option or the 'value.format' option are required. - - Required:required - - Type:String - - Default:(none) -
- -- **key.format** - - Description:The format used to deserialize and serialize the key part of Kafka messages. Please refer to the [formats](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) page for more details and more format options. Note: If a key format is defined, the 'key.fields' option is required as well. Otherwise the Kafka records will have an empty key. - - Required:optional - - Type:String - - Default:(none) -
- -- **key.fields** - - Description:Defines an explicit list of physical columns from the table schema that configure the data type for the key format. By default, this list is empty and thus a key is undefined. The list should look like 'field1;field2'. - - Required:optional - - Type:List - - Default:(none) -
- -- **key.fields-prefix** - - Description:Defines a custom prefix for all fields of the key format to avoid name clashes with fields of the value format. By default, the prefix is empty. If a custom prefix is defined, both the table schema and 'key.fields' will work with prefixed names. When constructing the data type of the key format, the prefix will be removed and the non-prefixed names will be used within the key format. Please note that this option requires that 'value.fields-include' must be set to 'EXCEPT_KEY'. - - Required:optional - - Type:String - - Default:(none) -
- -- **value.format** - - Description:The format used to deserialize and serialize the value part of Kafka messages. Please refer to the formats page for more details and more format options. Note: Either this option or the ['format'](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) option are required. - - Required:required - - Type:String - - Default:(none) -
- -- **value.fields-include** - - Description:Defines a strategy how to deal with key columns in the data type of the value format. By default, 'ALL' physical columns of the table schema will be included in the value format which means that key columns appear in the data type for both the key and value format. - - Required:optional - - Type:Enum - - Possible values: [ALL, EXCEPT_KEY] - - Default:ALL -
- -- **sink.partitioner** - - Description:String Output partitioning from Flink's partitions into Kafka's partitions. Valid values are default: use the kafka default partitioner to partition records. fixed: each Flink partition ends up in at most one Kafka partition. round-robin: a Flink partition is distributed to Kafka partitions sticky round-robin. It only works when record's keys are not specified. Custom FlinkKafkaPartitioner subclass: e.g. 'org.mycompany.MyPartitioner'.See the following [Sink Partitioning](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#sink-partitioning) for more details. - - Required:optional - - Type:String - - Default:default -
- -- **sink.semantic** - - Description:Defines the delivery semantic for the Kafka sink. Valid enumerationns are 'at-least-once', 'exactly-once' and 'none'. - See [Consistency guarantees](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#consistency-guarantees) for more details. - - Required:optional - - Type:String - - Default:at-least-once -
- -- **sink.parallelism** - - Description:Defines the parallelism of the Kafka sink operator. By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator. - - Required:optional - - Type:Integer - - Default:(none) -
- -## 5. Data Type - -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| no support | others | - -## 6. Example - -The details are in `chunjun-examples` dir. diff --git a/docs/connectors-en/kafka/kafka-source.md b/docs/connectors-en/kafka/kafka-source.md deleted file mode 100644 index a9c9a09684..0000000000 --- a/docs/connectors-en/kafka/kafka-source.md +++ /dev/null @@ -1,260 +0,0 @@ -# Kafka Source - -## 1. Introduce - -Kafka Source - -## 2. Version Support - -Kafka mainstream version - -## 3. Connector Name - -| Sync | kafkasource、kafkareader | -| --- | --- | -| SQL | kafka-x | - -## 4. Parameter description - -### 4.1. Sync - -- **topic** - - Description: Topic name of the Kafka record. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. When mode option is `timestamp` or `specific-offsets`, topic list isn't supported. - - Requested: required - - Type: String - - Default:(none) -
- -- **mode** - - Description: Startup mode for Kafka consumer, valid values are 'earliest-offset', 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. See the following [Start Reading Position](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html#start-reading-position) for more details. - - Requested: optional - - Type: String - - Default:group-offsets -
- -- **timestamp** - - Description: Start from user-supplied timestamp for each partition. only `kafkareader` - - Requested: It's requested, when the mode option is timestamp. - - Type: Long - - Default:(none) -
- -- **offset** - - Description: Start from user-supplied specific offsets for each partition. only `kafkareader` - - Requested: It's requested, when the mode option is timestamp. - - Type: String - - Format: partition:0,offset:42;partition:1,offset:300;partition:2,offset:300 - - Default:(none) -
- -- **groupId** - - Description: The id of the consumer group for Kafka. - - Requested: optional - - Type: String - - Default:default -
- -- **encoding** - - Description: character encoding - - Requested: optional - - Type: String - - Default:UTF-8 -
- -- **codec** - - Description: type of message format. Valid values are 'json', 'text'. - - text:It will put kafka record to a map, which map's key is 'message'. For example message in kafka is {"key":"key","message":"value"} - the data format sent to the downstream is: - ```json - [ - { - "message":"{\"key\": \"key\", \"value\": \"value\"}" - } - ] - ``` - - json:Treat message as JSON format - - If record include message key, the data format sent to the downstream is: - ```json - [ - { - "key1":"value1", - "message":"value" - } - ] - ``` - - When the message field isn't included, it will add a key and value. The data format sent to the downstream is: - ```json - [ - { - "key1":"value1", - "key2":"value2", - "message":"{\"key1\": \"value1\", \"key2\": \"value2\"}" - } - ] - ``` - - It will be treated as text, if message isn't JSON format. - - Requested: optional - - Type: String - - Default:text -
- -- **consumerSettings** - - Description: This can set and pass arbitrary Kafka configurations. It supports all options in `kafka.consumer.ConsumerConfig.ConsumerConfig` class. - - Requested: required - - Type: Map - - Default:(none) - - 如: - ```json - { - "consumerSettings":{ - "bootstrap.servers":"host1:9092,host2:9092,host3:9092" - } - } - ``` - -- **column** - - Description: Field type mapping for writer - - Requested: optional - - Type: List - - Default:(none) - - Note:each column options description: - - name:field name - - type:field type. It could be different from writer field type, ChunJun can auto convert. - - format: If field type is time attribute, It could be set time format, and auto convert from string type to timestamp type. - - for example: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss" - }] - ``` - -
- -### 4.2. SQL - -The details are in [kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) - -- **connector** - - Description:kafka-x - - Required:required - - Type:String - - Default:(none) -
- -- **topic** - - Description:Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified for sources. When the table is used as sink, the topic name is the topic to write data to. Note topic list is not supported for sinks. - - Required:required - - Type:String - - Default:(none) -
- -- **topic-pattern** - - Description:The regular expression for a pattern of topic names to read from. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. Note, only one of "topic-pattern" and "topic" can be specified for sources. - - Required:optional - - Type:String - - Default:(none) -
- -- **properties.bootstrap.servers** - - Description:Comma separated list of Kafka brokers. - - Required:required - - Type:String - - Default:(none) -
- -- **properties.group.id** - - Description:The id of the consumer group for Kafka source, optional for Kafka sink. - - Required:required by source - - Type:String - - Default:(none) -
- -- **properties.*** - - Description:This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in [Kafka Configuration documentation](https://kafka.apache.org/documentation/#configuration). Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. - - Required:optional - - Type:String - - Default:(none) -
- -- **format** - - Description: The format used to deserialize and serialize the value part of Kafka messages. Please refer to the [format](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) page for more details and more format options. Note: Either this option or the 'value.format' option are required. - - Required:required - - Type:String - - Default:(none) -
- -- **key.format** - - Description:The format used to deserialize and serialize the key part of Kafka messages. Please refer to the [formats](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) page for more details and more format options. Note: If a key format is defined, the 'key.fields' option is required as well. Otherwise the Kafka records will have an empty key. - - Required:optional - - Type:String - - Default:(none) -
- -- **key.fields** - - Description:Defines an explicit list of physical columns from the table schema that configure the data type for the key format. By default, this list is empty and thus a key is undefined. The list should look like 'field1;field2'. - - Required:optional - - Type:List - - Default:(none) -
- -- **key.fields-prefix** - - Description:Defines a custom prefix for all fields of the key format to avoid name clashes with fields of the value format. By default, the prefix is empty. If a custom prefix is defined, both the table schema and 'key.fields' will work with prefixed names. When constructing the data type of the key format, the prefix will be removed and the non-prefixed names will be used within the key format. Please note that this option requires that 'value.fields-include' must be set to 'EXCEPT_KEY'. - - Required:optional - - Type:String - - Default:(none) -
- -- **value.format** - - Description:The format used to deserialize and serialize the value part of Kafka messages. Please refer to the formats page for more details and more format options. Note: Either this option or the ['format'](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/) option are required. - - Required:required - - Type:String - - Default:(none) -
- -- **value.fields-include** - - Description:Defines a strategy how to deal with key columns in the data type of the value format. By default, 'ALL' physical columns of the table schema will be included in the value format which means that key columns appear in the data type for both the key and value format. - - Required:optional - - Type:Enum - - Possible values: [ALL, EXCEPT_KEY] - - Default:ALL -
- -- **scan.startup.mode** - - Description: Startup mode for Kafka consumer, valid values are 'earliest-offset', 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. See the following [Start Reading Position](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#start-reading-position) for more details. - - Requested: optional - - Type: String - - Default:group-offsets -
- -- **scan.startup.specific-offsets** - - Description: Specify offsets for each partition in case of 'specific-offsets' startup mode, e.g. 'partition:0,offset:42;partition:1,offset:300'. - - Requested: optional - - Type: String - - Default:(none) -
- -- **scan.startup.timestamp-millis** - - Description: Start from the specified epoch timestamp (milliseconds) used in case of 'timestamp' startup mode. - - Requested: optional - - Type: Long - - Default:(none) -
- -- **scan.topic-partition-discovery.interval** - - Description: Interval for consumer to discover dynamically created Kafka topics and partitions periodically. - - Requested: optional - - Type: Duration - - Default:(none) -
- -## 5. Data Type - -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| no support | others | - -## 6. Example - -The details are in `chunjun-examples` dir. diff --git a/docs/connectors-en/mongodb/mongodb-lookup.md b/docs/connectors-en/mongodb/mongodb-lookup.md deleted file mode 100644 index e8f889a045..0000000000 --- a/docs/connectors-en/mongodb/mongodb-lookup.md +++ /dev/null @@ -1,113 +0,0 @@ -# MongoDB Lookup -## 1. Introduce -MongoDB Lookup - -## 2. Version Support -MongoDB 3.4 and above - -## 3. Connector Name -| SQL | mongodb-x | -| --- | --- | - - - -## 4. Parameter description - -- **connector** - - Description:mongodb-x - - Required:required - - Default:(none) - - - -- **url** - - Description:URL of MongoDB connection,search [MongoDB Documents](https://docs.mongodb.com/manual/reference/connection-string/) for detail information. - - Required:optional - - Type:String - - Default:(none) - - - -- **collection** - - Description:collection name - - Required:required - - Default:(none) - - - -- **username** - - Description:user of database - - Required:optional - - Type:String - - Default:(none) - - - -- **password** - - Description:password of database - - Required:optional - - Type:String - - Default:(none) - - - -- **lookup.cache-type** - - Description:lookup table cache type(NONE、LRU、ALL) - - Required:optional - - Default:LRU - - - -- **lookup.cache-period** - - Description:time of interval ALL lookup table load data, Unit: ms. - - Required:optional - - Default:3600000 - - - -- **lookup.cache.max-rows** - - Description:size of data in lru lookup table cache. - - Required:optional - - Default:10000 - - - -- **lookup.cache.ttl** - - Description:time of data that lru lookup table cache. - - Required:optional - - Default:60000 - - - -- **lookup.fetch-size** - - Description:size of data that ALL lookup table load in every batch. - - Required:optional - - Default:1000 - - - -- **lookup.parallelism** - - Description:parallelism of lookup table. - - Required:optional - - Default:(none) - - - -## 5. Data Type -| support | int | -| --- | --- | -| | long | -| | double | -| | decimal | -| | objectId | -| | string | -| | bindata | -| | date | -| | timestamp | -| | bool | -| no support | array | - -## 6. Example -The details are in chunjun-examples dir. - - diff --git a/docs/connectors-en/mongodb/mongodb-sink.md b/docs/connectors-en/mongodb/mongodb-sink.md deleted file mode 100644 index 40be6563a5..0000000000 --- a/docs/connectors-en/mongodb/mongodb-sink.md +++ /dev/null @@ -1,180 +0,0 @@ -# MongoDB Sink -## 1. Introduce -MongoDB Sink - -## 2. Version Support -MongoDB 3.4 and above - - -## 3. Connector Name -| Sync | mongodbsink, mongodbwriter | -| --- | --- | -| SQL | mongodb-x | - - - -## 4. Parameter description -#### 4.1 Sync - -- **url** - - Description:URL of MongoDB connection,search [MongoDB Documents](https://docs.mongodb.com/manual/reference/connection-string/) for detail information. - - Required:optional - - Type:String - - Default:(none) - - - -- **hostPorts** - - Description:host and port of database, formatted like IP1:port. if using Multiple addresses, separated it by comma. - - Required:optional - - Type:String - - Default:(none) - - - -- **username** - - Description:user of database - - Required:optional - - Type:String - - Default:(none) - - - -- **password** - - Description:password of database - - Required:optional - - Type:String - - Default:(none) - - - -- **database** - - Description:name of database - - Required:required - - Type:String - - Default:(none) - - - -- **collectionName** - - Description:collection name of database - - Required:required - - Type:String - - Default:(none) - - - -- **replaceKey** - - Description:replaceKey specifies the primary key of each row of records, it's useful in replace and update mode. - - Required:optional - - Type:String - - Default:(none) - - - -- **writeMode** - - Description:write data mode, do not support replace and update mode when batchSize > 1. - - Required:optional - - Option:insert/replace/update - - Type:String - - Default:insert - - - -- **batchSize** - - Description:the size of rows in every single batch, it will decrease network communication with MongoDB, but a large number will cause system out of memory. - - Required:optional - - Type:int - - Default:1 - - - -- **flushIntervalMills** - - Description:time interval between each batch, Unit: ms. - - Required:optional - - Type:int - - Default:10000 - - - -### 2、SQL -SQL only support INSERT mode. in the future, we will support upsert mode if you configure the primary key. - -- **url** - - Description:URL of MongoDB connection,search [MongoDB Documents](https://docs.mongodb.com/manual/reference/connection-string/) for detail information. - - Required:optional - - Type:String - - Default:(none) - - - -- **database** - - Description:name of database - - Required:required - - Type:String - - Default:(none) - - - -- **collection** - - Description:collection name - - Required:required - - Default:(none) - - - -- **username** - - Description:user of database - - Required:optional - - Type:String - - Default:(none) - - - -- **password** - - Description:password of database - - Required:optional - - Type:String - - Default:(none) - - - -- **sink.parallelism** - - Description:parallelism of sink - - Required:optional - - Default:(none) - - - -- **sink.buffer-flush.max-rows** - - Description:the size of rows in every single batch - - Required:optional - - Default:(none) - - - -- **sink.buffer-flush.interval** - - Description:time interval between each batch, Unit: ms. - - Required:optional - - Default:(none) - - - -## 5. Type -| support | int | -| --- | --- | -| | long | -| | double | -| | decimal | -| | objectId | -| | string | -| | bindata | -| | date | -| | timestamp | -| | bool | -| no support | array | - - -## 6. Example -The details are in chunjun-examples dir. - diff --git a/docs/connectors-en/mongodb/mongodb-source.md b/docs/connectors-en/mongodb/mongodb-source.md deleted file mode 100644 index 225079daee..0000000000 --- a/docs/connectors-en/mongodb/mongodb-source.md +++ /dev/null @@ -1,111 +0,0 @@ -# MongoDB Source -## 1. Introduce -MongoDb Source - - -## 2. Version Support -MongoDB 3.4 and above - - -## 3. Connector Name -| Sync | mongodbsource、mongodbreader | -| --- | --- | -| SQL | mongodb-x | - - - -## 4. Parameter description -#### 4.1 Sync - -- **url** - - Description:URL of MongoDB connection,search [MongoDB Documents](https://docs.mongodb.com/manual/reference/connection-string/) for detail information. - - Required:optional - - Type:String - - Default:(none) - - - -- **hostPorts** - - Description:host and port of database, formatted like IP1:port. if using Multiple addresses, separated it by comma. - - Required:optional - - Type:String - - Default:(none) - - - -- **username** - - Description:user of database - - Required:optional - - Type:String - - Default:(none) - - - -- **password** - - Description:password of database - - Required:optional - - Type:String - - Default:(none) - - - -- **database** - - Description:name of database - - Required:required - - Type:String - - Default:(none) - - - -- **collectionName** - - Description:collection name of database - - Required:required - - Type:String - - Default:(none) - - - -- **fetchSize** - - Description:The number of data pieces read each time. Adjust this parameter to optimize the reading rate. Default 0 to let MongoDB Server choose the value itself. - - Required:optional - - Type:int - - Default:0 - - - -- **filter** - - Description:URL of MongoDB connection,search [MongoDB Documents](https://docs.mongodb.com/manual/reference/connection-string/) for detail information. - - Required:optional - - Type:String - - Default:(none) - - - -- **column** - - Description:columns that should be extract - - Notes: - - name:column name - - type:column type, It can be different from the field type in the database. - - Required:required - - Type:List - - Default:(none) -#### 4.2 SQL -do not support right now. - - -## 5. Data Type -| support | int | -| --- | --- | -| | long | -| | double | -| | decimal | -| | objectId | -| | string | -| | bindata | -| | date | -| | timestamp | -| | bool | -| no support | array | - -## 6. Example -The details are in chunjun-examples dir. diff --git a/docs/connectors-en/mysql/mysql-lookup.md b/docs/connectors-en/mysql/mysql-lookup.md deleted file mode 100644 index 51e8d9b2fe..0000000000 --- a/docs/connectors-en/mysql/mysql-lookup.md +++ /dev/null @@ -1,101 +0,0 @@ -# Mysql Lookup - -## 1. Introduction -Mysql Lookup table,full and asynchronous approaches are supported.
-Full cache: Load all dimensional table data into memory, and it is recommended that the amount of data is not used.
-Asynchronous caching: Query data using asynchronous means, and cache the queried data to memory using lru, which is recommended for large amounts of data. - -## 2. Support Version -mysql5.x - - -## 3. Plugin Name -| SQL | mysql-x | -| --- | --- | - -## 4. Parameter -- **connector** - - definition:mysql-x - - necessary:true - - data type:String - - default:null -
- -- **url** - - definition:jdbc:mysql://localhost:3306/test - - necessary:true - - data type:String - - default:null -
- -- **table-name** - - definition:the name of table - - necessary:true - - data type:String - - default:null: -
- -- **username** - - definition:username - - necessary:true - - data type:String - - default:null -
- -- **password** - - definition:password - - necessary:true - - data type:String - - default:null -
- -- **lookup.cache-type** - - definition:lookup table cache type(NONE、LRU、ALL) - - necessary:false - - data type:string - - default:LRU -
- -- **lookup.cache-period** - - definition:the time of interval to load data when use ALL lookup table, Unit: ms - - necessary:false - - data type:string - - default:3600000 -
- -- **lookup.cache.max-rows** - - definition:the size of data that lru lookup table cache - - necessary:false - - data type:string - - default:10000 -
- -- **lookup.cache.ttl** - - definition:the expire time of data that lru lookup table cache - - necessary:false - - data type:string - - default:60000 -
- -- **lookup.fetch-size** - - definition:the size of data that ALL lookup table load in every batch - - necessary:false - - data type:string - - default:1000 -
- -- **lookup.parallelism** - - definition:parallelism of lookup table - - necessary:false - - data type:string - - default:null -
- -## 5. Data type -| Support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| unSupport | ARRAY、MAP、STRUCT、UNION | - - -## 6. Profile Demo -see`chunjun-examples`directory. diff --git a/docs/connectors-en/mysql/mysql-sink.md b/docs/connectors-en/mysql/mysql-sink.md deleted file mode 100644 index 540a107ffc..0000000000 --- a/docs/connectors-en/mysql/mysql-sink.md +++ /dev/null @@ -1,214 +0,0 @@ -# Mysql Sink - -## 1. Introduction -mysql sink - -## 2. Support Version -mysql5.x - - -## 3. Plugin Name -| Sync | mysqlsink、mysqlwriter | -| --- | --- | -| SQL | mysql-x | - - -## 4. Parameter -### (1) Sync -- **connection** - - definition:Database connection parameters, including JDBC URL, schema, table and other parameters - - necessary:true - - data type:List - - default:null - ```text - "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - definition:JDBC connection string for relational database,search document for detail information of jdbcUrl:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - necessary:true - - data type:string - - default:null -
- -- **schema** - - definition:database schema name - - necessary:false - - data type:string - - default:null -
- -- **table** - - definition:the table name of the destination table. Currently only supports the configuration of a single table, and will support multiple tables in the future. - - necessary:true - - data type:List - - default:null -
- -- **username** - - definition:the user name of the data source - - necessary:true - - data type:String - - default:null -
- -- **password** - - definition:The password for the specified user name of the data source - - necessary:true - - data type:String - - default:null -
- -- **column** - - definition:The fields in the destination table that need to write data are separated by commas.eg."column": ["id","name","age"] - - necessary:true - - data type:List - - default:null -
- -- **fullColumn** - - definition:All fields in the destination table are separated by commas.eg. "column": ["id","name","age","hobby"],if not configured, it will be obtained in the system table - - necessary:false - - data type:List - - default:null -
- -- **preSql** - - definition:Before writing data to the destination table, a set of standard statements here will be executed first. - - necessary:false - - data type:List - - default:null -
- -- **postSql** - - definition:After writing data to the destination table, a set of standard statements here will be executed. - - necessary:false - - data type:List - - default:null -
- -- **writeMode** - - definition:Use insert into or replace into or ON DUPLICATE KEY UPDATE statement to control writing data to the target table. - - necessary:true - - options:insert/replace/update - - data type:String - - default:insert -
- -- **batchSize** - - definition:The size of the number of records submitted in batches at one time. This value can greatly reduce the number of network interactions between ChunJun and the database and improve the overall throughput. However, setting this value too large may cause the OOM situation of ChunJun running process. - - necessary:false - - data type:int - - default:1024 -
- -- **updateKey** - - definition:When the write mode is update and replace, you need to specify the value of this parameter as a unique index field - - Attention: - - If this parameter is empty and the write mode is update and replace, the application will automatically obtain the unique index in the database; - - If the data table does not have a unique index, but the write mode is configured as update and replace, the application will write the data in insert; - - necessary:false - - data type:Map - - eg."updateKey": {"key": ["id"]} - - default:null -
- -- **semantic** - - definition:Whether to enable two-phase commit on the sink side - - Attention: - - If this parameter is empty, two-phase commit is not enabled by default, that is, the sink side does not support exactly_once semantics; - - Currently only supports exactly-once and at-least-once - - necessary:false - - data type:String - - eg."semantic": "exactly-once" - - default:at-least-once -
- -### (2) SQL -- **connector** - - definition:mysql-x - - necessary:true - - data type:String - - default:null -
- -- **url** - - definition:jdbc:mysql://localhost:3306/test - - necessary:true - - data type:String - - default:null -
- -- **table-name** - - definition:the name of table - - necessary:true - - data type:String - - default:null: -
- -- **username** - - definition:username - - necessary:true - - data type:String - - default:null -
- -- **password** - - definition:password - - necessary:true - - data type:String - - default:null -
- -- **sink.buffer-flush.max-rows** - - definition:Number of batch write data,Unit: Piece - - necessary:false - - data type:String - - default:1024 -
- -- **sink.buffer-flush.interval** - - definition:Batch write interval,Unit: milliseconds - - necessary:false - - data type:String - - default:10000 -
- -- **sink.all-replace** - - definition:Whether to replace all the data in the database (if the original value in the database is not null, the new value is null, if it is true, it will be replaced with null) - - necessary:false - - data type:String - - default:false -
- -- **sink.parallelism** - - definition:Parallelism of writing results - - necessary:false - - data type:String - - default:null -
- -- **sink.semantic** - - definition:Whether to enable two-phase commit on the sink side - - Attention: - - If this parameter is empty, two-phase commit is not enabled by default, that is, the sink side does not support exactly_once semantics; - - Currently only supports exactly-once and at-least-once - - necessary:false - - data type:String - - eg."semantic": "exactly-once" - - default:at-least-once -
- -## 5. Data Type -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| unSupport | ARRAY、MAP、STRUCT、UNION | - - -## 6. Profile Demo -see`chunjun-examples`directory. diff --git a/docs/connectors-en/mysql/mysql-source.md b/docs/connectors-en/mysql/mysql-source.md deleted file mode 100644 index 3abc4891e3..0000000000 --- a/docs/connectors-en/mysql/mysql-source.md +++ /dev/null @@ -1,309 +0,0 @@ -# Mysql Source - -## 1. Introduction -Supports offline reading from mysql and real-time interval polling reading from mysql. - -## 2. Support Version -mysql5.x - - -## 3. Plugin Name -| Sync | mysqlsource、mysqlreader | -| --- | --- | -| SQL | mysql-x | - - -## 4. Parameter -### (1) Sync -- **connection** - - definition:Database connection parameters, including jdbcUrl, schema, table, and so on - - necessary:true - - data type:List - - default:null - ```text - "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - definition:The jdbc connection string for the relational database,search document for detail information of jdbcUrl:[MySQL doc](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - necessary:true - - data type:string - - default:null -
- -- **schema** - - definition:schema name of database - - necessary:false - - data type:string - - default:null -
- -- **table** - - definition:The name of the table of the destination table.Currently, only a single table is supported, and multiple tables are supported later. - - necessary:true - - data type:List - - default:null -
- -- **username** - - definition:username of database - - necessary:true - - data type:String - - default:null -
- -- **password** - - definition:password of database - - necessary:true - - data type:String - - default:null -
- -- **fetchSize** - - definition:Read the data size from the database at one time. MySQL will read all the results into the memory once by default. When the amount of data is large, it may cause OOM. Setting this parameter can control the fetchSize data read each time, instead of the default Read all the data at once; enable fetchSize to meet the following requirements: the database version must be higher than 5.0.2, and the connection parameter useCursorFetch=true. - Attention:The value of this parameter cannot be set too large, otherwise the reading will time out and the task will fail. - - necessary:false - - data type:int - - default:1024 -
- -- **where** - - definition:Filter conditions, the reader plug-in splices SQL according to the specified column, table, and where conditions, and performs data extraction based on this SQL. In actual business scenarios, the data of the day is often selected for synchronization, and the where condition can be specified as gmt_create> time. - - Attention:The where condition cannot be specified as limit 10. Limit is not a legal where clause of SQL. - - necessary:false - - data type:String - - default:null -
- -- **splitPk** - - definition:Specifying this parameter when channell in the speed configuration is greater than 1, the Reader plug-in stitches the sql based on the number of concurrings and the fields specified by this parameter, allowing each concurrent to read different data and increasing the read rate. - - Attention: - - SplitPk is recommended to use the table primary key, because the table primary key is usually more uniform, so the sliced out is not easy to appear data hot spots; - - Currently splitPk only supports integer data segmentation, and does not support other types such as floating point, string, and date. If the user specifies other non-supported types, ChunJun will report an error; - - If the channel is greater than 1 but this parameter is not configured, the task will be set as failed. - - necessary:false - - data type:String - - default:null -
- -- **queryTimeOut** - - definition:Query timeout,Unit: second。 - - Attention:When the amount of data is large, or when querying from a view, or a custom sql query, you can specify the timeout period through this parameter. - - necessary:false - - data type:int - - default:1000 -
- -- **customSql** - - definition:For custom query statements, if only the specified fields cannot meet the requirements, you can specify the query sql through this parameter, which can be arbitrarily complex query statements. - - Attention: - - It can only be a query statement, otherwise it will cause the task to fail; - - The fields returned by the query statement need to correspond to the fields in the column list; - - When this parameter is specified, the table specified in the connection is invalid; - - When specifying this parameter, column must specify specific field information, and cannot be replaced by *; - - necessary:false - - data type:String - - default:null -
- -- **column** - - definition:Need to read the field. - - format:Support 3 formats -
1.Read all fields, if there are a lot of fields, you can use the following wording: - - ```bash - "column":["*"] - ``` - 2.Specify only the field name: - - ``` - "column":["id","name"] - ``` - 3.Specify specific information: - - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - Property description: - - name:the name of the field - - type:The field type can be different from the field type in the database, the program will do a type conversion - - format:If the field is a time string, you can specify the time format and convert the field type to date format to return - - value:If the specified field does not exist in the database, the value of value will be returned as a constant column. If the specified field exists, when the value of the specified field is null, the value will be returned as default. - - necessary:true - - default:null -
- -- **polling** - - definition:Whether to enable interval polling, after enabling it, data will be periodically pulled from the database according to the pollingInterval polling interval. To enable interval polling, you need to configure the parameters pollingInterval and increColumn, and you can choose the configuration parameter startLocation. If the parameter startLocation is not configured, the maximum value of the incremental field will be queried from the database as the starting position of the poll when the task starts. - - necessary:false - - data type:Boolean - - default:false -
- -- **pollingInterval** - - definition:Polling interval, the interval of pulling data from the database, the default is 5000 milliseconds. - - necessary:false - - data type:long - - default:5000 -
- -- **increColumn** - - definition:Incremental field, which can be the corresponding incremental field name, or a pure number, indicating the sequential position of the incremental field in the column (starting from 0) - - necessary:false - - data type:String or int - - default:null -
- -- **startLocation** - - definition:Incremental query start position - - necessary:false - - data type:String - - default:null -
- -- **useMaxFunc** - - definition:It is used to mark whether to save one or more pieces of data of the endLocation location, true: do not save, false (default): save, in some cases the last few data may be recorded repeatedly, this parameter can be configured as true. - - necessary:false - - data type:Boolean - - default:false -
- -- **requestAccumulatorInterval** - - definition:The interval between sending the query accumulator request. - - necessary:false - - data type:int - - default:2 -
- -### (2) SQL -- **connector** - - definition:mysql-x - - necessary:true - - data type:String - - default:null -
- -- **url** - - definition:jdbc:mysql://localhost:3306/test - - necessary:true - - data type:String - - default:null -
- -- **table-name** - - definition:table-name - - necessary:true - - data type:String - - default:null: -
- -- **username** - - definition:username - - necessary:true - - data type:String - - default:null -
- -- **password** - - definition:password - - necessary:true - - data type:String - - default:null -
- -- **scan.polling-interval** - - definition:Interval training time.Optional(Leave blank as patch task),default value is null. - - necessary:false - - data type:String - - default:null -
- -- **scan.parallelism** - - definition:Parallelism - - necessary:false - - data type:String - - default:null -
- -- **scan.fetch-size** - - definition:Each fetch size from the database.Unit: Piece - - necessary:false - - data type:String - - default:1024 -
- -- **scan.query-timeout** - - definition:Database connection timeout time, unit: second. - - necessary:false - - data type:String - - default:1 -
- -- **scan.partition.column** - - definition:The segmentation field used when multiple parallelism is enabled to read data - - necessary:false - - data type:String - - default:null -
- -- **scan.partition.strategy** - - definition:Data fragmentation strategy - - necessary:false - - data type:String - - default:range -
- -- **scan.increment.column** - - definition:Increment field name - - necessary:false - - data type:String - - default:null -
- -- **scan.increment.column-type** - - definition:Incremental field type - - necessary:false - - data type:String - - default:null -
- -- **scan.start-location** - - definition:The start position of the increment field, if not specified, all will be synchronized first, and then in the increment - - necessary:false - - data type:String - - default:null -
- -- **scan.restore.columnname** - - definition:When check-point is turned on, the task continues with the field name of save-point/check-point. If you continue to run, it will overwrite the start position of scan.start-location, starting from the point where you continue to run. - - necessary:false - - data type:String - - default:null -
- -- **scan.restore.columntype** - - definition:When check-point is turned on, the task continues from save-point/check-point field type - - necessary:false - - data type:String - - default:null -
- -## 5. Data Type -| Support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| unSupport | ARRAY、MAP、STRUCT、UNION | - - -## 6. Profile Demo -see`chunjun-examples`directory. - diff --git a/docs/connectors-en/oracle/oracle-lookup.md b/docs/connectors-en/oracle/oracle-lookup.md deleted file mode 100644 index 2741d9176d..0000000000 --- a/docs/connectors-en/oracle/oracle-lookup.md +++ /dev/null @@ -1,104 +0,0 @@ -# Oracle Lookup - -## 1、Introduce -Oracle lookup,support all and lru cache
-all cache:All data would be loaded into memory since the program start ,which is not recommended to use in scenarios with large amount of data .
-lru cache:Query data asynchronously and add data to lru cache,which is recommended to use in scenarios with large amount of data. - -## 2、Version Support -Oracle 9 and above - - -## 3、Connector name -| SQL | oracle-x | -| --- | --- | - -## 4、Parameter description -- **connector** - - Description:oracle-x - - Required:optional - - Type:String - - Default:none -
- -- **url** - - Description:jdbc:oracle:thin:@0.0.0.1:1521:orcl - - Required:required - - Type:String - - Default:none -
- -- **table-name** - - Description:table name - - Required:required - - Type:String - - Default:none -
- -- **username** - - Description:username - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none -
- -- **lookup.cache-type** - - Description:lookup table type (NONE、LRU、ALL),default value is LRU - - Required:optional - - Type:String - - Default:LRU -
- -- **lookup.cache-period** - - Description:Interval for loading data when the cache type is all,default value is 3600000ms - - Required:optional - - Type:string - - Default:3600000 -
- -- **lookup.cache.max-rows** - - Description:the cache rows of lru lookup table ,default value is 10000 - - Required:optional - - Type:string - - Default:10000 -
- -- **lookup.cache.ttl** - - Description:Interval for loading data when the cache type is lru,default value is 60000ms - - Required:optional - - Type:string - - Default:60000 -
- -- **lookup.fetch-size** - - Description:the num of data fetched from the oracle table which is used as lookup all table at a time - - Required:optional - - Type:string - - Default:1000 -
- -- **lookup.parallelism** - - Description:the parallelism of the lookup table - - Required:optional - - Type:string - - DEfault:none -
- -## 5、Supported data type -| Supported data type | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| Not supported at the moment | BFILE、XMLTYPE、Collections、BLOB、CLOB、NCLOB | - -Attention:Oracle numeric data may lose precision during conversion due to the limit of flink DecimalType's PRECISION(1~38) and SCALE(0~PRECISION) - - - -## 六、Demo -see details in `chunjun-examples` dir of project chunjun. diff --git a/docs/connectors-en/oracle/oracle-sink.md b/docs/connectors-en/oracle/oracle-sink.md deleted file mode 100644 index d35e9c7dad..0000000000 --- a/docs/connectors-en/oracle/oracle-sink.md +++ /dev/null @@ -1,217 +0,0 @@ -# Oracle Sink - -## 1、Introduce -oracle sink - -## 2、Version Support -Oracle 9 and above - - -## 3、Connector name -| Sync | oraclesink、oraclewriter | -| --- | --- | -| SQL | oracle-x | - - -## 4、Parameter description -### 1、Sync -- **connection** - - Description:param for Database connection,including jdbcUrl、schema、table and so on - - Required:required - - Type:List - - Default:none - ```text - "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - Description:jdbc url,See for details[Oracle](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) - - Required:required - - Type:string - - Default:none -
- -- **schema** - - Description:Database schema - - Required:optional - - Type:string - - Default:oracle user name -
- -- **table** - - Description: oracle table name, only support one table in a single work at the moment. - - Required:required - - Type:List - - Default:none -
- -- **username** - - Description: user name - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description: password - - Required:required - - Type:String - - Default:none -
- -- **column** - - Description:the fields to be written to the destination table,which is separated by English commas.for example: "column": ["id","name","age"] - - Required:required - - Type:List - - Default:none -
- -- **fullcolumn** - - Description:All fields in the destination table ,which is separated by English commas.for example: "column": ["id","name","age","hobby"],if not configured, it will be obtained in the system table - - Required:optional - - Type:List - - Default:none -
- -- **preSql** - - Description:the sql executed before writing data into the destination table - - Required:optional - - Type:List - - Default:none -
- -- **postSql** - - Description:the sql executed after writing data into the destination table - - Required:optional - - Type:List - - Default:none -
- -- **writeMode** - - Description:the mode of writing data, insert into or merge into - - Required:required - - All options:insert/update - - Type:String - - Default:insert -
- -- **batchSize** - - Description:The number of records submitted in batch at one time. This value can greatly reduce the number of network interactions between chunjun and the database and improve the overall throughput,Setting this value too large may cause the chunjun process to run oom - - Required:optional - - Type:int - - Default:1024 -
- -- **updateKey** - - Description:When the write mode is update, you need to specify the value of this parameter as the unique index field - - attention: - - If this parameter is empty and the write mode is update, the application will automatically obtain the unique index in the database; - - If the data table does not have a unique index, but the required write mode is configured as update and, the application will write data in the way of insert; - - Required:optional - - Type:Map - - for example:"updateKey": {"key": ["id"]} - - Default:none -
- -- **semantic** - - Description:sink operator support phase two commit - - attention: - -If this parameter is blank, phase two commit is not enabled by default,which means sink operators do not support exactly-once semantics - -Currently only supported exactly-once and at-least-once - - Required:optional - - Type:String - - for example:"semantic": "exactly-once" - - Default:at-least-once -
- -### 2、SQL -- **connector** - - Description:oracle-x - - Required:required - - Type:String - - Default:none -
- -- **url** - - Description:jdbc:oracle:thin:@0.0.0.1:1521:orcl - - Required:required - - Type:String - - Default:none -
- -- **table-name** - - Description: table name - - Required:required - - Type:String - - Default:none: -
- -- **username** - - Description:username - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none -
- -- **sink.buffer-flush.max-rows** - - Description:Number of data pieces written in batch - - Required:optional - - Type:String - - Default:1024 -
- -- **sink.buffer-flush.interval** - - Description:Batch write interval,Unit: ms - - Required:optional - - Type:String - - Default:10000 -
- -- **sink.all-replace** - - Description: whether to replace all data in the database - - Required:optional - - Type:String - - Default:false -
- -- **sink.parallelism** - - Description:the parallelism of sink operator - - Required:optional - - Type:String - - Default:none -
- -- **sink.semantic** - - Description:sink operator support phase two commit - - attention: - -If this parameter is blank, phase two commit is not enabled by default,which means sink operators do not support exactly-once semantics; - -Currently only supported exactly-once and at-least-once - - Required:optional - - Type:String - - for example:"semantic": "exactly-once" - - Default:at-least-once -
- -## 5、Supported data type -| Supported data type | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| Not supported at the moment | BFILE、XMLTYPE、Collections、BLOB、CLOB、NCLOB | - -Attention:Oracle numeric data may lose precision during conversion due to the limit of flink DecimalType's PRECISION(1~38) and SCALE(0~PRECISION) - - - -## 6、Demo -see details in `chunjun-examples` dir of project chunjun. diff --git a/docs/connectors-en/oracle/oracle-source.md b/docs/connectors-en/oracle/oracle-source.md deleted file mode 100644 index ce308d97cc..0000000000 --- a/docs/connectors-en/oracle/oracle-source.md +++ /dev/null @@ -1,302 +0,0 @@ -# Oracle Source - -## 1、Introduce -read data from oracle with batch model - -## 2、Version Support -Oracle 9 and above - - -## 3、Connector name -| Sync | oraclesource、oraclereader | -| --- | --- | -| SQL | oracle-x | - - -## 4、Parameter description -### 1、Sync -- **connection** - - Description:param for Database connection,including jdbcUrl、schema、table and so on - - Required:required - - Type:List - - Default:none - ```text - "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - Description:jdbc url,See for details[Oracle](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) - - Required:required - - Type:string - - Default:user name -
- -- **schema** - - Description:Database schema - - Required:optional - - Type:string - - Default:none -
- -- **table** - - Description:oracle table name, only support one table in a single work at the moment - - Required:required - - Type:List - - Default:none -
- -- **username** - - Description:user name - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none -
- -- **fetchSize** - - Description:the num of records fetching from Oracle at one time,the default value is 1024.When fetchsize is set too small, frequent data reading will affect query speed and database pressure. When fetchsize is set too large, it may cause oom when the amount of data is large. Setting this parameter can control fetchsize pieces of data to be read each time. - Attention:The value of this parameter cannot be set too large, otherwise it will read timeout and cause the task to fail. - - Required:optional - - Type:int - - Default:1024 -
- -- **where** - - Description:query condition,readerThe plugin splices SQL according to the specified column, table and where conditions,In the actual business scenario, the data of the current day is often selected for synchronization, and the where condition can be specified as GMT_ create > time. - - Attention:The where condition cannot be specified as limit . Limit is not a legal where clause of SQL. - - Required:optional - - Type:String - - Default:none -
- -- **splitPk** - - Description:When the channel in the speed configuration is greater than 1, this parameter is specified. The reader plug-in splices SQL according to the number of concurrencies and the fields specified by this parameter, so that each concurrency can read different data and improve the reading rate. - - Attention: - - It is recommended that splitpk use the table primary key, because the table primary key is usually uniform, so the segmented fragments are not prone to data hot spots. - - At present, splitpk only supports integer data segmentation and does not support other types such as floating point, string and date. If the user specifies other unsupported types, chunjun will report an error. - - If the channel is greater than 1 but this parameter is not configured, the task will be set as failed - - Required:optional - - Type:String - - Default:none -
- -- **queryTimeOut** - - Description:Query timeout, unit seconds - - Attention:When there is a large amount of data, or query from the view, or custom SQL query, you can specify the timeout through this parameter. - - Required:optional - - Type:int - - Default:1000 -
- -- **customSql** - - Description:For user-defined query statements, if only specified fields cannot meet the requirements, you can specify the SQL of the query through this parameter, which can be any complex query statement. - - Attention: - - Only query statements can be used, otherwise the task will fail; - - The fields returned by the query statement need to correspond to the fields in the column list; - - When specifying this parameter, column must specify specific field information and cannot be replaced by * sign; - - Required:optional - - Type:String - - Default:none -
- -- **column** - - Description:Fields to read. - - Format:Three formats are supported -
1.Read all fields. If there are many fields, you can use the following writing method: - ```bash - "column":["*"] - ``` - 2.Specify only field names: - ``` - "column":["id","name"] - ``` - 3.Specify specific information: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - Attribute description: - - name:Field name - - type:Field type,It can be different from the field type in the database. The program will make a type conversion - - format:If the field is a time string, you can specify the format of time and convert the field type to date format - - value:If the specified field does not exist in the database, the value will be returned as a constant column. If the specified field exists, when the value of the specified field is null, the value will be returned as default - - Required:required - - Default:none -
- -- **polling** - - Description:Whether to enable interval polling. After it is enabled, data will be periodically pulled from the database according to the pollinginterval polling interval. To enable interval polling, you also need to configure the parameters pollinginterval and increcolumn. You can select the configuration parameter startlocation. If the parameter startlocation is not configured, the maximum value of the increment field will be queried from the database as the starting position of polling when the task is started. - - Required:optional - - Type:Boolean - - Default:false -
- -- **pollingInterval** - - Description:Polling interval: the interval between pulling data from the database. The default is 5000 milliseconds. - - Required:optional - - Type:long - - Default:5000 -
- -- **increColumn** - - Description:The incremental field can be the corresponding incremental field name or a pure number, indicating the sequential position of the incremental field in the column (starting from 0) - - Required:optional - - Type:String或int - - Default:none -
- -- **startLocation** - - Description:Start position of incremental query - - Required:optional - - Type:String - - Default:none -
- -- **useMaxFunc** - - Description:Used to mark whether to save one or more pieces of data at endlocation. True: do not save, false (default): save. In some cases, the last few pieces of data may be repeatedly recorded. You can configure this parameter to true - - Required:optional - - Type:Boolean - - Default:false -
- -- **requestAccumulatorInterval** - - Description:The interval between sending query accumulator requests - - Required:optional - - Type:int - - Default:2 -
- -### 2、SQL -- **connector** - - Description:oracle-x - - Required:required - - Type:String - - Default:none -
- -- **url** - - Description:jdbc:oracle:thin:@0.0.0.1:1521:orcl - - Required:required - - Type:String - - Default:none -
- -- **table-name** - - Description:table name - - Required:required - - Type:String - - Default:none: -
- -- **username** - - Description:username - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none -
- -- **scan.polling-interval** - - Description:Interval rotation training time. It is not required (it is not filled in as an offline task). - - Required:optional - - Type:String - - Default:none -
- -- **scan.parallelism** - - Description:parallelism - - Required:optional - - Type:String - - Default:none -
- -- **scan.fetch-size** - - Description:Size of each fetch from the database. - - Required:optional - - Type:String - - Default:1024 -
- -- **scan.query-timeout** - - Description:The size of each fetch from the database, unit: database connection timeout, unit: seconds. - - Required:optional - - Type:String - - Default:1 -
- -- **scan.partition.column** - - Description:The segmentation field read by multiple parallelism must be set under multiple parallelism - - Required:optional - - Type:String - - Default:none -
- -- **scan.partition.strategy** - - Description:Data fragmentation strategy - - Required:optional - - Type:String - - Default:range -
- -- **scan.increment.column** - - Description:Incremental field name - - Required:optional - - Type:String - - Default:none -
- -- **scan.increment.column-type** - - Description:Incremental field type - - Required:optional - - Type:String - - Default:none -
- -- **scan.start-location** - - Description:The start position of the increment field. If it is not specified, all are synchronized first, and then in the increment field - - Required:optional - - Type:String - - Default:none -
- -- **scan.restore.columnname** - - Description:When CP is enabled, the task starts from the SP / CP continuation field name. If you continue, the start position of scan.start-location will be overwritten, starting from the continuation point - - Required:optional - - Type:String - - Default:none -
- -- **scan.restore.columntype** - - Description:When CP is enabled,Task continuation field type from SP / CP - - Required:optional - - Type:String - - Default:none -
- -## 5、Supported data type -| Supported data type | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| Not supported at the moment | BFILE、XMLTYPE、Collections、BLOB、CLOB、NCLOB | - -Attention:Oracle numeric data may lose precision during conversion due to the limit of flink DecimalType's PRECISION(1~38) and SCALE(0~PRECISION) diff --git a/docs/connectors-en/pgwal/PostgresCDC-Source-en.md b/docs/connectors-en/pgwal/PostgresCDC-Source-en.md deleted file mode 100644 index 7235369428..0000000000 --- a/docs/connectors-en/pgwal/PostgresCDC-Source-en.md +++ /dev/null @@ -1,141 +0,0 @@ -# Postgres CDC Source - - - - -- [Ⅰ、Introduction](#Ⅰ、Introduction) -- [Ⅱ、Supported Versions](#Ⅱ、Supported Versions) -- [Ⅲ、Plugin name](#Ⅲ、Plugin name) -- [Ⅳ、Database Configuration](#Ⅳ、Database Configuration) -- [Ⅴ、Parameter description](#Ⅴ、Parameter description) - - [1 、Sync](#1sync) - - [2 、SQL](#2sql) -- [Ⅵ、Data Type](#Ⅵ、Data Type) -- [Ⅶ、Script example](#Ⅶ、Script example) - - - -## Ⅰ、Introduction -The Postgres CDC plugin captures change data from Postgres in real time. Currently, the sink plugin does not support data restoration, and can only write changed log data. - -## Ⅱ、Supported version -Postgres 10.0+ - -## Ⅲ、Plugin name -| Sync | pgwalsource、pgwalreader | -| --- | --- | -| SQL | pgwal-x | - -## Ⅳ、Database Configuration -1. The write-ahead log level (wal_level) must be logical -2. The plugin is implemented based on the PostgreSQL logical replication and logical decoding functions, so the PostgreSQL account has at least replication permissions, and if it is allowed to create slots, it has at least super administrator permissions -3. For detailed principles, please refer to the official PostgreSQL documentation http://postgres.cn/docs/10/index.html - - - -## Ⅴ、Parameter description -### 1、Sync - -- **url** - - Description:JDBC URL link for Postgresql - - Required:yes - - Parameter type: string - - Default value: none - -
- -- **username** - - Description: username - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **password** - - Description: Password - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **databaseName** - - Description:the database name - - Required:yes - - Parameter type:string - - Default value:none - -
- -- **tableList** - - Description:List of tables to be parsed - - Notice:After specifying this parameter, the filter parameter will be invalid, the table and filter are empty, listen to all tables under the schema in jdbcUrl - - Required:no - - Parameter type:list - - Default value:none - -
- -- **slotName** - - Description:slot name - - Required:no - - Parameter type:String - - Default value:true - -
- -- **allowCreated** - - Description:Whether to automatically create a slot - - Required:no - - Parameter type:boolean - - Default value:false - -
- -- **temporary** - - Description:Whether it is a temporary slot - - Required:no - - Parameter type:boolean - - Default value:false - -
- -- **statusInterval** - - Description:Heartbeat interval - - Required:no - - Parameter type:int - - Default value:10 - -
- -- **lsn** - - Description:Log sequence number - - Required:no - - Parameter type:long - - Default value:0 - -
- -- **slotAvailable** - - Description:Is the slot available - - Required:no - - Parameter type:boolean - - Default value:false - -
- - -## Ⅵ、Data Type -| Support | BIT | -| --- | --- | -| | NULL、 BOOLEAN、 TINYINT、 SMALLINT、 INTEGER、 INTERVAL_YEAR_MONTH、 BIGINT| -| | INTERVAL_DAY_TIME、 DATE、 TIME_WITHOUT_TIME_ZONE | -| | TIMESTAMP_WITHOUT_TIME_ZONE、 TIMESTAMP_WITH_LOCAL_TIME_ZONE、 FLOAT | -| | DOUBLE、 CHAR、 VARCHAR、 DECIMAL、 BINARY、 VARBINARY | -| Not supported yet | none | - - -## Ⅶ、Script example -See the `chunjun-examples` folder in the project. - diff --git a/docs/connectors-en/redis/redis-lookup.md b/docs/connectors-en/redis/redis-lookup.md deleted file mode 100644 index e823456b81..0000000000 --- a/docs/connectors-en/redis/redis-lookup.md +++ /dev/null @@ -1,139 +0,0 @@ -# Redis Lookup - -## 一、Introduce -Redis dimension table supports full and async methods
-Full cache: Load all dimension table data into memory. It is recommended to use it with a small amount of data.
-Async cache: Query data asynchronously and cache the queried data into memory using LRU. It is recommended to use it with a large amount of data. - -## 二、Supported Version -All major versions - - -## 三、Plugin Name -| SQL | redis-x | -| --- | --- | - -## 四、Configuration -- **connector** - - Description:redis-x - - Required:yes - - Type:string - - Default:(none) -
- -- **url** - - Description:localhost:6379 - - Required:yes - - Type:string - - Default:(none) -
- -- **table-name** - - Description:tableName - - Required:yes - - Type:string - - Default:(none) -
- -- **password** - - Description:password - - Required:no - - Type:string - - Default:(none) -
- -- **redis-type** - - Description: Redis mode(1 standalone,2 sentinel, 3 cluster) - - Required:no - - Type:string - - Default:1 -
- -- **master-name** - - Description: Master node name (required in sentinel mode) - - Required:no - - Type:string - - Default:(none) -
- -- **database** - - Description: Redis database index - - Required:no - - Type:string - - Default:0 -
- -- **timeout** - - Description:Connection timeout - - Unit:millisecond - - Required:no - - Type:string - - Default:10000 -
- -- **max.total** - - Description:Maximum connection - - Required:no - - Type:string - - Default:8 -
- -- **max.idle** - - Description:Maximum number of free connections - - Required:no - - Type:string - - Default:8 -
- -- **min.idle** - - Description: Minimum number of free connections - - Required:no - - Type:string - - Default:0 -
- -- **lookup.cache-type** - - Description:Dimension table cache type (NONE、LRU、ALL) - - Required:no - - Type:string - - Default:LRU -
- -- **lookup.cache-period** - - Description:How often do all dimension tables load data - - Unit:millisecond - - Required:no - - Type:string - - Default:3600000 -
- -- **lookup.cache.max-rows** - - Description:Number of entries of LRU dimension table cache data - - Required:no - - Type:string - - Default:10000 -
- -- **lookup.cache.ttl** - - Description:LRU dimension table cache data time - - Unit:millisecond - - Required:no - - Type:string - - Default:60000 -
- -- **lookup.parallelism** - - Description:parallelism - - Required:no - - Type:string - - Default:(none) -
- -## 五、Data Types -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| not support | ARRAY、MAP、STRUCT、UNION | - - -## 六、examples -link `chunjun-examples` diff --git a/docs/connectors-en/redis/redis-sink.md b/docs/connectors-en/redis/redis-sink.md deleted file mode 100644 index d21fa8bba0..0000000000 --- a/docs/connectors-en/redis/redis-sink.md +++ /dev/null @@ -1,208 +0,0 @@ -# Redis Sink - -## 一、Introduce -Redis sink - -## 二、Supported Version -All major versions - - -## 三、Plugin Name -| Sync | redissink、rediswriter | -| --- | --- | -| SQL | redis-x | - - -## 四、Configuration -### 1、Sync -- **hostPort** - - Description:The Redis address and port - - Required:yes - - Default:localhost:6379 -
- -- **password** - - Description:The Redis password - - Required:yes - - Default:(none) -
- -- **database** - - Description:The Redis database - - Required:no - - Default:0 -
- -- **keyFieldDelimiter** - - Description:Write the key separator of redis. If multiple keys need to be spliced, this value is required. If there is only one key, this configuration item can be ignored. For example: key = key1 \ u0001id - - Required:no - - Default:\u0001 -
- -- **dateFormat** - - Description:When writing to redis, the time format of date - - Required:no - - Default:(none) -
- -- **expireTime** - - Description:Redis value cache expiration time (if it needs to be permanently valid, this configuration item can be omitted) - - Required:no - - Default:0 -
- -- **timeout** - - Description:Timeout for writing to redis - - Unit:millisecond - - Required:no - - Default:30000 -
- -- **type和mode** - - Description:Type indicates the data type and mode indicates the write mode - - Options:string/list/set/zset/hash - - | Type | Mode | - | ---- | ---- | - | string | set | - | list | lpush | - | list | rpush | - | set | sadd | - | zset | zadd | - | hash | hset | - - Required:yes - - Default:(none) -
- -- **valueFieldDelimiter** - - Description:This configuration item considers that when each row of the source data exceeds two columns (if your source data has only two columns, namely key and value, you can ignore this configuration item and do not fill it in). When the value type is string, the separator between values, such as value1 \ u0001value2 \ u0001value3 - - Required:no - - Default:\u0001 -
- -- **keyIndexes** - - Description:Keyindexes indicates which columns on the source side need to be used as keys (the first column starts from 0). If the first and second columns need to be combined as keys, the value of keyindexes is [0,1] - - Note:After configuring keyindexes, redis writer will take the remaining columns as values. If you only want to synchronize some columns of the source table as keys and some columns as values, you don't need to synchronize all fields, you can specify columns on the reader plug-in side and filter the columns. For example, the data in redis is "test, redis, first, second", keyindexes = [0,1], so the obtained key is "test \ \ u0001redis", and the value is "first \ \ u0001second" - - Required:yes - - Default:(none) -
- -### 2、SQL -- **connector** - - Description:redis-x - - Required:yes - - Type:string - - Default:(none) -
- -- **url** - - Description:localhost:6379 - - Required:yes - - Type:string - - Default:(none) -
- -- **table-name** - - Description:tableName - - Required:yes - - Type:string - - Default:(none) -
- -- **password** - - Description:password - - Required:no - - Type:string - - Default:(none) -
- -- **redis-type** - - Description: Redis mode(1 standalone,2 sentinel, 3 cluster) - - Required:no - - Type:string - - Default:1 -
- -- **master-name** - - Description: Master node name (required in sentinel mode) - - Required:no - - Type:string - - Default:(none) -
- -- **database** - - Description: Redis database index - - Required:no - - Type:string - - Default:0 -
- -- **timeout** - - Description:Connection timeout - - Unit:millisecond - - Required:no - - Type:string - - Default:10000 -
- -- **max.total** - - Description:Maximum connection - - Required:no - - Type:string - - Default:8 -
- -- **max.idle** - - Description:Maximum number of free connections - - Required:no - - Type:string - - Default:8 -
- -- **min.idle** - - Description: Minimum number of free connections - - Required:no - - Type:string - - Default:0 -
- -- **key.expired-time** - - Description:The expiration time of the key of redis sink. The default is 0 (never expires) - - Unit:second - - Required:no - - Type:string - - Default:0 -
- -- **sink.parallelism** - - Description:Sink parallelism - - Required:no - - Type:string - - Default:(none) -
- -- **type和mode** - - Description:Type indicates the data type and mode indicates the write mode - - Options:string/list/set/zset/hash - - | Type | Mode | - | ---- | ---- | - | string | set | - | list | lpush | - | list | rpush | - | set | sadd | - | zset | zadd | - | hash | hset | - - Required:yes - - Default:(none) -
- - -## 五、Data Types -| support | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| ---------- | --- | -| not support | ARRAY、MAP、STRUCT、UNION | - - -## 六、examples -link `chunjun-examples` diff --git a/docs/connectors-en/sqlserver/sqlserver-lookup.md b/docs/connectors-en/sqlserver/sqlserver-lookup.md deleted file mode 100644 index 2d948ee1c2..0000000000 --- a/docs/connectors-en/sqlserver/sqlserver-lookup.md +++ /dev/null @@ -1,119 +0,0 @@ -# SqlServer Lookup - -## 1. Introduction -SqlServer Lookup table,full and asynchronous approaches are supported.
-Full cache: Load all dimensional table data into memory, and it is recommended that the amount of data is not used.
-Asynchronous caching: Query data using asynchronous means, and cache the queried data to memory using lru, which is recommended for large amounts of data. - -## 2. Support Version -Microsoft SQL Server 2012 and above - -## 3. Plugin Name -| SQL | sqlserver-x | -| --- | --- | - -## 4. Parameter - -- **connector** - - definition:connection plugin name - - required:required - - data type:String - - value:sqlserver-x - -​
- -- **url** - - definition:Use the open source jtds driver connection instead of Microsoft's official driver
jdbcUrl Reference documents:[jtds Reference documents](http://jtds.sourceforge.net/faq.html) - - required:required - - data type:String - - default:none - -
- -- **table-name** - - definition:table name - - required:required - - data type:String - - default:none - -
- -- **schema** - - definition:schema - - required:optional - - data type:String - - default:none - -​
- -- **username** - - definition:username - - required:required - - data type:String - - default:none - -​
- -- **password** - - definition:password - - required:required - - data type:String - - default:none - -​
- -- **lookup.cache-type** - - definition:Dimension table cache type(NONE、LRU、ALL),default is LRU - - required:optional - - data type:string - - default:LRU - -
- -- **lookup.cache-period** - - definition:Interval for loading data when the cache type is all, default is 3600000ms(one hour) - - required:optional - - data type:string - - default:3600000 - -
- -- **lookup.cache.max-rows** - - definition:the cache rows of lru lookup table ,default value is 10000 - - required:optional - - data type:string - - default:10000 - -
- -- **lookup.cache.ttl** - - definition:Interval for loading data when the cache type is lru,default value is 60000ms - - required:optional - - data type:string - - default:60000 - -​
- -- **lookup.fetch-size** - - definition:the num of data fetched from the oracle table which is used as lookup all table at a time - - required:optional - - data type:string - - default:1000 - -​
- -- **lookup.parallelism** - - definition:the parallelism of the lookup table - - required:optional - - data type:string - - default:none - - - -## 5. Data type -|Supported data type | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | -| ---| ---| -| Not supported at the moment | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | - -## 6. Profile Demo -see `chunjun-examples` directory。 diff --git a/docs/connectors-en/sqlserver/sqlserver-sink.md b/docs/connectors-en/sqlserver/sqlserver-sink.md deleted file mode 100644 index fcfa7c0c12..0000000000 --- a/docs/connectors-en/sqlserver/sqlserver-sink.md +++ /dev/null @@ -1,254 +0,0 @@ -# SqlServer Source - -## 1. Introduce -SqlServer Sink support to write data to database SQLServer - -## 2. Version Support -Microsoft SQL Server 2012 and above - -## 3、Connector name -| Sync | sqlserverwriter、sqlserversink | -| --- | --- | -| SQL | sqlserver-x | - -## 4、Parameter description - -### 1.Sync - -- **connection** - - Description:param for Database connection,including jdbcUrl、schema、table and so on - - Required:required - - Type:List - - Default:none - -```json -"connection": [{ - "jdbcUrl": "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack", - "table": ["table"], - "schema":"public" - }] -``` - - - -
- -- **jdbcUrl** - - Description:Use the open source jtds driver connection instead of Microsoft's official driver
jdbcUrlReference documents:[jtds Reference documents](http://jtds.sourceforge.net/faq.html) - - Required:required - - Type:String - - Default:none - - -
- -- **schema** - - Description:Database schema - - Required:optional - - Type:string - - Default: none -
- -- **table** - - Description: sqlserver table name, only support one table in a single work at the moment. - - Required:required - - Type:List - - Default:none -
- -- **username** - - Description: user name - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description: password - - Required:required - - Type:String - - Default:none -
- -- **column** - - Description:the fields to be written to the destination table,which is separated by English commas.for example: "column": ["id","name","age"] - - Required:required - - Type:List - - Default:none -
- - -
- -- **fullcolumn** - - Description:All fields in the destination table ,which is separated by English commas.for example: "column": ["id","name","age","hobby"],if not configured, it will be obtained in the system table - - Required:optional - - Type:List - - Default:none -
- -- **preSql** - - Description:the sql executed before writing data into the destination table - - Required:optional - - Type:List - - Default:none -
- -​
- -- **postSql** - - Description:the sql executed after writing data into the destination table - - Required:optional - - Type:List - - Default:none -
- -
- -- **writeMode** - - Description:the mode of writing data, insert into or merge into - - Required:required - - All options:insert/update - - Type:String - - Default:insert - -
- -- **withNoLock** - - Description:add the sql with(nolock) - - Required:optional - - Type:Boolean - - Default:false - -
- -- **updateKey** - - Description:When the write mode is update, you need to specify the value of this parameter as the unique index field - - attention: - - If this parameter is empty and the write mode is update, the application will automatically obtain the unique index in the database; - - If the data table does not have a unique index, but the required write mode is configured as update and, the application will write data in the way of insert; - - Required:optional - - Type:Map - - for example:"updateKey": {"key": ["id"]} - - Default:none - - -
- -- **semantic** - - Description:sink operator support phase two commit - - attention: - -If this parameter is blank, phase two commit is not enabled by default,which means sink operators do not support exactly-once semantics - -Currently only supported exactly-once and at-least-once - - Required:optional - - Type:String - - for example:"semantic": "exactly-once" - - Default:at-least-once - -- **batchSize** - - Description:The number of records submitted in batch at one time. This value can greatly reduce the number of network interactions between chunjun and the database and improve the overall throughput,Setting this value too large may cause the chunjun process to run oom - - Required:optional - - Type:int - - Default:1024 - - - -### 2.SQL - -- **connector** - - Description:connector type - - Required:required - - Type:String - - value:sqlserver-x - -
- -- **url** - - Description:Use the open source jtds driver connection instead of Microsoft's official driver - - Required:required - - Type:String - - Default:none - -​
- -- **table-name** - - Description: table name - - Required:required - - Type:String - - Default:none -
- -- **schema** - - Description:schema - - Required:optional - - Type:String - - Default:none - -
- -- **username** - - Description:username - - Required:required - - Type:String - - Default:none -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none -
- -
- -- **sink.buffer-flush.max-rows** - - Description:Number of data pieces written in batch - - Required:optional - - Type:String - - Default:1024 -
- - -- **sink.buffer-flush.interval** - - Description:Batch write interval,Unit: ms - - Required:optional - - Type:String - - Default:10000 -
- -- **sink.all-replace** - - Description: whether to replace all data in the database - - Required:optional - - Type:String - - Default:false -
- -
- -- **sink.semantic** - - Description:sink operator support phase two commit - - attention: - -If this parameter is blank, phase two commit is not enabled by default,which means sink operators do not support exactly-once semantics; - -Currently only supported exactly-once and at-least-once - - Required:optional - - Type:String - - for example:"semantic": "exactly-once" - - Default:at-least-once -
- -- **sink.parallelism** - - Description:the parallelism of sink operator - - Required:optional - - Type:String - - Default:none -
- - - -## 5. Supported data type -|Supported data type | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | -| ---| ---| -| Not supported at the moment | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | -## 6. Demo -see details in`chunjun-examples` directory. diff --git a/docs/connectors-en/sqlserver/sqlserver-source.md b/docs/connectors-en/sqlserver/sqlserver-source.md deleted file mode 100644 index 60d7f13b3c..0000000000 --- a/docs/connectors-en/sqlserver/sqlserver-source.md +++ /dev/null @@ -1,355 +0,0 @@ -# SqlServer Source - -## 1. Introduction -SqlServer Source plugin support read data from database SQLServer

- -## 2. Support Version -Microsoft SQL Server 2012 and above

- -## 3. Plugin Name - -| Sync | sqlserverreader、sqlserversource | -| --- | --- | -| SQL | sqlserver-x | - -## 4. Parameter - -### 1.Sync - -- **connection** - - Description:param for Database connection,including jdbcUrl、schema、table and so on - - Required:required - - Type:List - - Default:none -```json -"connection": [{ - "jdbcUrl": ["jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack"], - "table": ["table"], - "schema":"public" - }] -``` - - -
- -- **jdbcUrl** - - Description:Use the open source jtds driver connection instead of Microsoft's official driver
jdbcUrlReference documents:[jtds Reference documents](http://jtds.sourceforge.net/faq.html) - - Required:required - - type:string - - default:none - - -​
- -- **schema** - - Description:Database schema - - Required:optional - - Type:string - - Default:none - - -
- -- **table** - - Description: table name, only support one table in a single work at the moment - - Required:required - - Type:List - - Default:none - - -
-- **username** - - Description:user name - - Required:required - - Type:String - - Default:none - - -
- -- **password** - - Description:password - - Required:required - - Type:String - - Default:none - - -
- -- **where** - - Description:query condition,readerThe plugin splices SQL according to the specified column, table and where conditions,In the actual business scenario, the data of the current day is often selected for synchronization, and the where condition can be specified as GMT_ create > time. - - Attention:The where condition cannot be specified as limit . Limit is not a legal where clause of SQL. - - Required:optional - - Type:String - - Default:none - -
- -- **splitPk** - - Description:When the channel in the speed configuration is greater than 1, this parameter is specified. The reader plug-in splices SQL according to the number of concurrencies and the fields specified by this parameter, so that each concurrency can read different data and improve the reading rate. - - Attention: - - It is recommended that splitpk use the table primary key, because the table primary key is usually uniform, so the segmented fragments are not prone to data hot spots. - - At present, splitpk only supports integer data segmentation and does not support other types such as floating point, string and date. If the user specifies other unsupported types, chunjun will report an error. - - If the channel is greater than 1 but this parameter is not configured, the task will be set as failed - - Required:optional - - Type:String - - Default:none - - -
- -- **fetchSize** - - Description:the num of records fetching from database at one time,the default value is 1024.When fetchsize is set too small, frequent data reading will affect query speed and database pressure. When fetchsize is set too large, it may cause oom when the amount of data is large. Setting this parameter can control fetchsize pieces of data to be read each time. - Attention:The value of this parameter cannot be set too large, otherwise it will read timeout and cause the task to fail. - - Required:optional - - Type:int - - Default:1024 - - -
- -- **queryTimeOut** - - Description:Query timeout, unit seconds - - Attention:When there is a large amount of data, or query from the view, or custom SQL query, you can specify the timeout through this parameter. - - Required:optional - - Type:int - - Default:1000 - - -
- -- **customSql** - - Description:For user-defined query statements, if only specified fields cannot meet the requirements, you can specify the SQL of the query through this parameter, which can be any complex query statement. - - Attention: - - Only query statements can be used, otherwise the task will fail; - - The fields returned by the query statement need to correspond to the fields in the column list; - - When specifying this parameter, column must specify specific field information and cannot be replaced by * sign; - - Required:optional - - Type:String - - Default:none - - -
- -- **withNoLock** - - 描述:是否在sql语句后面添加 with(nolock) - - 必选:否 - - 字段类型:Boolean - - 默认值:false - - -
- -- **column** - - Description:Fields to read. - - Format:Three formats are supported -
1.Read all fields. If there are many fields, you can use the following writing method: - ```bash - "column":["*"] - ``` - 2.Specify only field names: - ``` - "column":["id","name"] - ``` - 3.Specify specific information: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - Attribute description: - - name:Field name - - type:Field type,It can be different from the field type in the database. The program will make a type conversion - - format:If the field is a time string, you can specify the format of time and convert the field type to date format - - value:If the specified field does not exist in the database, the value will be returned as a constant column. If the specified field exists, when the value of the specified field is null, the value will be returned as default - - Required:required - - Default:none - -
- -- **polling** - - Description:Whether to enable interval polling. After it is enabled, data will be periodically pulled from the database according to the pollinginterval polling interval. To enable interval polling, you also need to configure the parameters pollinginterval and increcolumn. You can select the configuration parameter startlocation. If the parameter startlocation is not configured, the maximum value of the increment field will be queried from the database as the starting position of polling when the task is started. - - Required:optional - - Type:Boolean - - Default:false - - -
- -- **pollingInterval** - - Description:Polling interval: the interval between pulling data from the database. The default is 5000 milliseconds. - - Required:optional - - Type:long - - Default:5000 - - -
- -- **increColumn** - - Description:The incremental field can be the corresponding incremental field name or a pure number, indicating the sequential position of the incremental field in the column (starting from 0) - - Required:optional - - Type:String或int - - Default:none - - -
- -- **startLocation** - - Description:Start position of incremental query - - Required:optional - - Type:String - - Default:none - - -
- -- **useMaxFunc** - - Description:Used to mark whether to save one or more pieces of data at endlocation. True: do not save, false (default): save. In some cases, the last few pieces of data may be repeatedly recorded. You can configure this parameter to true - - Required:optional - - Type:Boolean - - Default:false -
- -- **requestAccumulatorInterval** - - Description:The interval between sending query accumulator requests - - Required:optional - - Type:int - - Default:2 - -### 2.SQL - -- **connector** - - definition:connector type - - required: - - type:String - - value:sqlserver-x - -
- -- **url** - - definition:Use the open source jtds driver connection instead of Microsoft's official driver - - required:required - - type:String - - default:none - -​
- - -- **schema** - - definition:schema - - required:optional - - type:String - - default:none - -​
- -- **table-name** - - definition:table-name - - required:required - - type:String - - default:none: -
- -- **username** - - definition:username - - required:required - - type:String - - default:none -
- -- **password** - - definition:password - - required:required - - type:String - - default:none -
- -- **scan.polling-interval** - - definition:Interval training time.Optional(Leave blank as patch task),default value is null. - - required:optional - - type:String - - default:none -
- -- **scan.parallelism** - - definition:Parallelism - - required:optional - - type:String - - default:none -
- -- **scan.fetch-size** - - definition:Each fetch size from the database.Unit: Piece - - required:optional - - data type:String - - default:1024 -
- -- **scan.query-timeout** - - definition:Database connection timeout time, unit: second. - - required:optional - - type:String - - default:1 -
- -- **scan.partition.column** - - definition:The segmentation field used when multiple parallelism is enabled to read data - - required:optional - - type:String - - default:none -
- -- **scan.partition.strategy** - - definition:Data fragmentation strategy - - required:optional - - type:String - - default:range -
- -- **scan.increment.column** - - definition:Increment field name - - required:optional - - type:String - - default:none -
- -- **scan.increment.column-type** - - definition:Incremental field type - - required:optional - - type:String - - default:none -
- -- **scan.start-location** - - definition:The start position of the increment field, if not specified, all will be synchronized first, and then in the increment - - required:optional - - type:String - - default:none -
- -- **scan.restore.columnname** - - definition:When check-point is turned on, the task continues with the field name of save-point/check-point. If you continue to run, it will overwrite the start position of scan.start-location, starting from the point where you continue to run. - - required:optional - - type:String - - default:none -
- -- **scan.restore.columntype** - - definition:When check-point is turned on, the task continues from save-point/check-point field type - - required:optional - - type:String - - default:none -
- - -## 5. Data Type - -|Support | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | -| ---| ---| -| unSupport | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | - -# 6. Profile Demo - see`chunjun-examples`directory. diff --git a/docs/connectors-en/sqlservercdc/SqlserverCDC-Principles.md b/docs/connectors-en/sqlservercdc/SqlserverCDC-Principles.md deleted file mode 100644 index b0df98667c..0000000000 --- a/docs/connectors-en/sqlservercdc/SqlserverCDC-Principles.md +++ /dev/null @@ -1,284 +0,0 @@ -# SqlServer CDC Real-Time Collection Principles - - - -- [SqlServer CDC Real-Time Collection Principle](#sqlserver-cdc Real-Time Collection Principle) -- [I. Fundamentals](# a foundation) -- [II. Configuration](# II configuration) -- [III. Principle](# III principle) - - [1. SQL Server Agent](#1sql-server-agent) - - [2. Database CDC before and after opening](#2 database cdc before and after opening) - - [3. business table CDC before and after opening](#3business table CDC before and after opening) - - [4. collection principle](#4 collection principle) - - [1. insert/delete](#1insertdelete) - - [2. update](#2update) - - [3. Flowchart](#3flowchart) - - [4. data format](#4data format) - - - -# I. Basics - -SqlServer official support for CDC from SqlServer 2008 version, the document link is as follows. -[https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-ver15](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-ver15) - -# II. Configuration - -The configuration document link is as follows. -[SqlServerConfigurationCDC](./other/SqlserverCDC configuration.md) - -# Three, the principle - -### 1. SQL Server Agent - -SQL Server Agent agent service, a standard service of sql server, serves as an agent to perform all sql automation tasks, as well as unattended tasks such as database transactional replication. This service is stopped in the default installation, you need to start it manually or change it to automatic movement, otherwise the automation tasks of sql will not be executed, also pay attention to the service startup account. -Simply put, this service is started before the capture process will process the transaction log and write entries to the CDC table. -[https://docs.microsoft.com/zh-cn/sql/ssms/agent/sql-server-agent?view=sql-server-ver15](https://docs.microsoft.com/zh-cn/sql/ssms/agent/sql-server-agent?view=sql-server-ver15) - -### 2. Comparison of database CDC before and after turning it on - -Before turning on. - -
- -
-
-After opening. -EXEC sys.sp_cdc_enable_db; - -
- -
-
- -We first observe that a new **systranschemas** table has been added under dbo. The **systranschemas** table is used to track architectural changes in projects published in transaction releases and snapshot releases. - - -| Column Name | Data Type | Description | -| ------------- | ----------- | ------------------------------------------------------------- | -| tabid | int | Identifies the table item where the schema change occurred. | -| startlsn | binary | The LSN value at the start of the schema change. | -| endlsn | binary | The LSN value at the end of the schema change. | -| typeid | int | The type of architecture change. | - -A new schema named cdc has been added under the database, and in fact a new cdc user has been added. cdc has the following four new tables. -
-**1. captured_columns** -Returns a row for each column tracked in the captured instance. By default, all columns in the source table will be captured. However, if the source table is enabled for change data capture, columns can be included in or excluded from the capture by specifying a list of columns. -When there is no business table with CDC enabled, the table is empty. - - -| Column Name | Data Type | Description | -| ---------------- | ----------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| object_id | int | The ID of the change table to which the captured column belongs. | -| column_name | sysname | The name of the captured column. | -| column_id | int | The ID of the captured column in the source table. | -| column_type | sysname | The type of the captured column. | -| column_ordinal | int | Change the column ordinal number in the table (starting from 1). The metadata columns in the change table will be excluded. Serial number 1 will be assigned to the first column captured. | -| is_computed | bit | Indicates that the captured column is a computed column from the source table. | - -**2. change_tables** -Returns one row for each changed table in the database. When change data capture is enabled for a source table, a change table is created. -When no business table has CDC enabled, the table is empty. - - -| Column Name | Data Type | Description | -| ---------------------- | ------------ | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| object_id | int | The ID of the change table. Unique in the database. | -| version | int | Identified for reference only. Not supported. Future compatibility is not guaranteed. For SQL Server 2012 (11.x), this column always returns 0. | -| source_object_id | int | The ID of the source table enabled for change data capture. | -| capture_instance | sysname | The name of the capture instance used to name the instance-specific trace object. By default, the name is derived from the source schema name plus the source table name in the format schemaname_sourcename. | -| start_lsn | binary(10) | Log sequence number (LSN), indicating the low endpoint when querying the change data in the change table. null = low endpoint not yet established. | -| end_lsn | binary(10) | Identified for reference only. Not supported. Future compatibility is not guaranteed. For SQL Server 2008, this column always returns NULL. | -| supports_net_changes | bit | Enables query net change support for change tables. | -| has_drop_pending | bit | The capture process received a notification that the source table has been deleted. | -| role_name | sysname | The name of the database role used to access the changed data. nULL = role not used. | -| index_name | sysname | The name of the index used to uniquely identify the rows in the source table. index_name is the name of the primary key index of the source table, or the name of the unique index specified when change data capture was enabled for the source table. NULL = The source table had no primary key and no unique index was specified when change data capture was enabled. Note: If change data capture is enabled for a table with a primary key, the Change Data Capture feature will use the index regardless of whether net changes are enabled. When change data capture is enabled, changes to the primary key will not be allowed. If the table does not have a primary key, change data capture can still be enabled, but only net changes can be set to False. Once change data capture is enabled, primary keys can be created. Since the change data capture feature does not use primary keys, you can also modify primary keys. | -| filegroup_name | sysname | Changes the name of the filegroup in which the table resides. NULL = Changes the default filegroup in which the table resides in the database. | -| create_date | datetime | The date on which the source table was enabled. | -| partition_switch | bit | Indicates whether the SWITCH PARTITION command of ALTER TABLE can be executed on a table with change data capture enabled. 0 indicates that partition switching is blocked. Unpartitioned tables always return 1. | - -**3. ddl_history** -Returns one row for each Data Definition Language (DDL) change made to a table with change data capture enabled. You can use this table to determine when DDL changes occurred to the source table and what was changed. This table does not contain any entries for the source table where no DDL changes occurred. -This table is empty when there are no changes to the table structure of any business table with CDC turned on. - - -| Column Name | Data Type | Description | -| ------------------------ | --------------- | ------------------------------------------------------------------------------------------------------------------------------------------------- | -| source_object_id | int | The ID of the source table to which the DDL change was applied. | -| object_id | int | The ID of the changed table associated with the capture instance of the source table. | -| required_column_update | bit | Indicates that the data type of the capture column was modified in the source table. This modification changes the columns in the change table. | -| ddl_command | nvarchar(max) | The DDL statement to be applied to the source table. | -| ddl_lsn | binary(10) | The log sequence number (LSN) associated with the commit of the DDL modification. | -| ddl_time | datetime | The date and time of the DDL changes made to the source table. | - -**4. index_columns** -Returns one row for each index column associated with the changed table. Change data capture uses these index columns to uniquely identify the rows in the source table. By default, the primary key columns of the source table will be included. However, if the source table's unique index is specified when change data capture is enabled for the source table, the columns in that index will be used instead. If net change tracking is enabled, the primary key or unique index is required for that source table. -When no index column exists for any business table with CDC enabled, the table is empty. - - -| Column Name | Data Type | Description | -| --------------- | ----------- | ---------------------------------------------------------- | -| object_id | int | Change the ID of the table. | -| column_name | sysname | The name of the index column. | -| index_ordinal | tinyint | The number of the column in the index (starting from 1). | -| column_id | int | The ID of the column in the source table. | - -**5. lsn_time_mapping** -Returns a row for each transaction that has a row in the change table. This table is used to create a mapping between the log sequence number (LSN) commit value and the time of the committed transaction. Items that do not have corresponding change table entries can also be logged so that the table records the completion of LSN processing during periods of little or no change activity. - - -| Column Name | Data Type | Description | -| ----------------- | ---------------- | ---------------------------------------------------------------- | -| start_lsn | binary(10) | The LSN of the committed transaction. | -| tran_begin_time | datetime | The time when the transaction associated with the LSN started. | -| tran_end_time | datetime | The time when the transaction ended. | -| tran_id | varbinary (10) | The ID of the transaction. | - -The following functions have been added under cdc. -
- -**1. fn_cdc_get_all_changes_** -Returns one row for each change applied to the source table within the specified log sequence number (LSN) range. If the source row has multiple changes within that interval, each change is represented in the returned result set. In addition to returning change data, the four metadata columns provide the information needed to apply changes to another data source. The row filter option controls the content of the metadata columns and the rows returned in the result set. When the "all" row filter option is specified, there will be only one row for each change to identify that change. When the "all update old" option is specified, the update operation is represented as two rows: one containing the values of the captured columns before the update and the other containing the values of the captured columns after the update. This enumerated function is created when change data capture is enabled for the source table. The function name is derived and uses the **cdc.fn_cdc_get_all_changes_**_capture_instance_ format, where _capture_instance_ is the value specified for the capture instance when change data capture is enabled for the source table. - - -| column_name | data_type | description | -| ---------------------------------- | ---------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| __$start_lsn | binary(10) | The commit LSN associated with the change, used to preserve the commit order of the change. Changes committed in the same transaction will share the same commit LSN value. | -| __$seqval | binary(10) | Sequence of values used to sort the row changes within a transaction. | -| __$operation | int | Identifies the Data Manipulation Language (DML) operation required to apply the changed data row to the target data source. Can be one of the following values.
1 = Delete
2 = Insert
3 = Update (the captured column value is the value before the update operation is performed). This value is applied only if the row filter option "all update old" is specified.
4 = update (the captured column value is the value after the update operation is performed). | -| __$update_mask | varbinary(128) | A bit mask that corresponds to one bit for each captured column that identifies the capture instance. When\__\$operation = 1 or 2, this value sets all defined bits to 1. When __$ operation = 3 or 4, only the bit corresponding to the changed column is set to 1. | -| \ | various | The remaining columns returned by the function are the captured columns identified at the time the capture instance was created. If no columns are specified in the list of captured columns, all columns in the source table are returned. | - -**2. fn_cdc_get_net_changes_** -Returns a net changes row for each source row within the specified log sequence number in the (LSN) range, in the same format as above. - -### 3. Comparison of before and after business table CDC is turned on - -Before opening is the same as the previous chart - -Open SQL. - -```sql -sys.sp_cdc_enable_table --- the name of the schema to which the table belongs -[ @source_schema = ] 'source_schema', - --- the name of the table -[ @source_name = ] 'source_name' , - --- is the name of the database role used to control access to the changed data -[ @role_name = ] 'role_name' - --- is the name of the capture instance used to name the change data capture object, a name that will need to be used frequently in later stored procedures and functions. -[,[ @capture_instance = ] 'capture_instance' ] - --- Indicates whether net change query support is enabled for this capture instance if this table has a primary key or a unique index that has been identified using the @index_name parameter. otherwise, this parameter defaults to 0. -[,[ @supports_net_changes = ] supports_net_changes ] - --- The name of the unique index used to uniquely identify the rows in the source table. index_name is sysname and can be NULL. --- If specified, index_name must be the only valid index of the source table. If index_name is specified, the index column identified takes precedence over any defined primary key column, just like the table's unique row identifier. -[,[ @index_name = ] 'index_name' ] - --- Which columns need to be captured. captured_column_list has the data type nvarchar(max) and can be NULL. if NULL, all columns will be included in the change table. -[,[ @captured_column_list = ] 'captured_column_list' ] - --- is the filegroup to be used for the changes table created for the capture instance. -[,[ @filegroup_name = ] 'filegroup_name' ] - --- Indicates whether the SWITCH PARTITION command of ALTER TABLE can be executed on tables with change data capture enabled. --- allow_partition_switch is bit, default value is 1. -[,[ @partition_switch = ] 'partition_switch' ] -``` - -When turned on. - -
- -
-
-At this point, a new table named dbo_kudu_CT is added under cdc, and for any business table with CDC turned on, a table of format \${schema}\_${table}\_CT will be created under its corresponding cdc schema. - -**1. dbo_kudu_CT:** -The change table created when change data capture is enabled for the source table. This table returns one row for each insert and delete operation performed on the source table and two rows for each update operation performed on the source table. If the name of the change table is not specified when the source table is enabled, a derived name is used. The format of the name is cdc. capture_instance _CT where capture_instance is the name of the source table's schema and the name of the source table in the format schema_table. For example, if change data capture is enabled for the table Person in the AdventureWorks sample database, the derived change table name will be cdc.Person_Address_CT. - - -| Column Name | Data Type | Description | -| ---------------------------------- | ----------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| __$start_lsn | binary(10) | The log sequence number (LSN) associated with the commit transaction for the corresponding change. All changes committed in the same transaction will share the same commit LSN. For example, if a delete operation on a source table deletes two rows, the changes table will contain two rows, each with the same __ $ start_lsn value. | -| __ $ end_lsn | binary(10) | Marked as for information only. Not supported. Future compatibility is not guaranteed. In SQL Server 2012 (11.x), this column is always NULL. | -| __$seqval | binary(10) | Sequential value used to sort row changes within a transaction. | -| __$operation | int | Identifies the Data Manipulation Language (DML) operation associated with the corresponding change. Can be one of the following values.
1 = Delete
2 = Insert
3 = Update (old value) column data with the row value before the update statement was executed.
4 = Update (new value) column data with row values after the update statement is executed. | -| __$update_mask | varbinary(128) | Bitmask based on the column serial number of the change table, used to identify those columns that have changed. | -| \ | Many and varied | The remaining columns in the change table are those that were identified as captured columns in the source table when the capture instance was created. If no columns are specified in the list of captured columns, all columns in the source table will be included in this table. | -| __ $ command_id | int | Tracks the order of operations in a transaction. | - -**2. captured_columns:** - -
- -
-
- -**3. change_tables:** - -
- -
- -
- -### 4. Acquisition principle! - -#### 1. insert/delete - -For insert and delete type of data changes, for each row change will add a row in the corresponding \${schema}\_${table}\_CT table. For insert, id, user_id, and name are the value values after insert; for delete, id, user_id, and name are the value values before delete. - -
- -
- -
- -#### 2. update - -a. Primary key updated -At this time, the SqlServer database practice is within the same thing, the original record is deleted first, and then re-inserted. -Execute the following SQL, log table as shown in the figure. -UPDATE [dbo]. [kudu] SET [id] = 2, [user_id] = '2', [name] = 'b' WHERE [id] = 1; - -
- -
-
-b. Primary key not updated -At this point, what the SqlServer database does is to update the field information directly. -Execute the following SQL with the log table as shown in the figure. -UPDATE [dbo]. [kudu] SET [user_id] = '3', [name] = 'c' WHERE [id] = 2; - -
- -
-
- -#### 3. Flowchart - -
- -
-
-For ChunJun SqlServer CDC real-time collection plug-in, the basic principle is to call the fn_cdc_get_all_changes_ function in a polling manner, cyclically, to get the data between the last ending lsn and the current maximum lsn value of the database. For insert/delete types, it gets and parses one row, and for update types, it gets and parses two rows. After parsing, pass the data to the downstream and record the lsn of the current parsed data to prepare for the next polling. - -#### 4. Data format - -```json -{ - "type": "update", - "schema": "dbo", - "table": "tb1", - "lsn": "00000032:0000002038:0005", - "ts": 6760525407742726144, - "before_id":1, - "after_id":2 -} -``` diff --git a/docs/connectors-en/sqlservercdc/SqlserverCDC-configuration.md b/docs/connectors-en/sqlservercdc/SqlserverCDC-configuration.md deleted file mode 100644 index d5834433d8..0000000000 --- a/docs/connectors-en/sqlservercdc/SqlserverCDC-configuration.md +++ /dev/null @@ -1,142 +0,0 @@ -# SqlServer Configuring CDC - - - -- [SqlServer configuration CDC](#sqlserver configuration cdc) - - [1. query SqlServer database version](#1 query sqlserver database version) - - [2. query the current user rights, must be a member of the sysadmin fixed server role to allow the database to enable the CDC (change data capture) function](#2 query the current user rights must be a member of the sysadmin fixed server role to allow the database to enable the cdc change data capture function) - - [3. query whether the database has been enabled CDC (change data capture) function](#3 query whether the database has been enabled cdc change data capture function) - - [4. Enable CDC(change data capture) function for database database](#4Enable CDC change data capture function for database database) - - [5. query whether the table has enabled CDC (change data capture) function](#5 query whether the table has enabled cdc change data capture function) - - [6. Enable CDC(change data capture) function for table](#6 enable CDC change data capture function for table) - - [7. confirm whether CDC agent is started normally](#7 confirm whether cdc-agent-is started normally) - -Note: SqlServer has supported CDC (Change Data Capture) since version 2008, this article is based on SqlServer 2017. - - - -#### 1. Query SqlServer database version - -SQL: `SELECT @@VERSION` -Result. - -
- -
- -#### 2. Query the current user permissions, must be a member of sysadmin fixed server role to allow the CDC (Change Data Capture) feature to be enabled on the database - -SQL: `exec sp_helpsrvrolemember 'sysadmin'` -Result. - -
- -
- -#### 3. Query whether the database has enabled CDC (Change Data Capture) function - -SQL: `select is_cdc_enabled, name from sys.databases where name = 'tudou'` -Result. - -
- -
-
-0: not enabled; 1: enabled - -#### 4. Enable CDC (Change Data Capture) for database databases - -SQL. - -```sql -USE tudou - GO -EXEC sys.sp_cdc_enable_db -GO -``` - -Repeat step 3 to make sure the database is enabled for CDC (Change Data Capture). - -
- -
- -#### 5. Query whether the table has enabled CDC (Change Data Capture) function - -SQL: `select name,is_tracked_by_cdc from sys.tables where name = 'test';` -Result. - -
- -
- - - -At this point, the table `test` starts the CDC (Change Data Capture) function configuration is complete. - -#### 7. Verify that the CDC agent is started properly - -```sql -EXEC master.dbo.xp_servicecontrol N'QUERYSTATE', N'SQLSERVERAGENT' -``` - -
- -
-If the above status is displayed, you need to start the corresponding agent. - -**Windows environment operation to enable CDC agent** -Click on the following location to enable the agent - -
- -
- -**Restart the database** - -
- -
- -**Check the agent status again to confirm the status change to running** - -
- -
- -At this point, the table `test` starts the CDC (change data capture) function configuration is complete. - -**docker environment operation to turn on CDC agent** - -**Enable agent service for mssql-server**_ - -```shell -docker exec -it sqlserver bash -/opt/mssql/bin/mssql-conf set sqlagent.enabled true -docker stop sqlserver -docker start sqlserver -``` - -Reference reading: [https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-2017](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-2017) diff --git a/docs/connectors-en/sqlservercdc/SqlserverCDC-source.md b/docs/connectors-en/sqlservercdc/SqlserverCDC-source.md deleted file mode 100644 index 996cbd00a6..0000000000 --- a/docs/connectors-en/sqlservercdc/SqlserverCDC-source.md +++ /dev/null @@ -1,284 +0,0 @@ -# SqlserverCDC Source - - - -- [I. Introduction](# I Introduction) -- [II. Supported Versions](# II Supported Versions) -- [III. Database Configuration](# III Database Configuration) -- [IV, SqlserverCDC principle] (# IV SqlserverCDC principle) -- [V, plug-in name] (# V plug-in name) -- [VI, parameter description] (# VI parameter description) - - [1, Sync](#1sync) - - [2, SQL](#2sql) -- vii. data structure](#seven data structure) -- [viii, data types](#viii data types) -- [ix, script example](#nine script example) - - - -
- -## I. Introduction - -The Sqlservercdc plugin supports configuring the listener table name as well as reading the starting point to read log data. SQLservercdc saves the currently consumed lsn at checkpoint, so the support runs continuously. -
- -## II. Supported Versions - -SqlServer 2012, 2014, 2016, 2017, 2019 standalone -
- -## III. Database configuration - -[SqlserverCDC configuration](SqlserverCDC configuration.md) -
- -## IV. SqlserverCDC Principle - -[SqlserverCDC Principle](SqlserverCDC Principle.md) -
- -## V. Plugin Name - -| Sync | sqlservercdcreader, sqlservercdcsource | -| ---- | -------------------------------------- | -| SQL | sqlservercdc-x | - -
- -## - -## VI. Parameter Description - -### 1. Sync - -- **url** - - Description: JDBC URL link for sqlserver database - - Required: yes - - Parameter type: string - - Default value: none - -
- - -- **username** - - Description: username - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **password** - - Description: Password - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **tableList** - - Description: The tables to listen to, e.g. ["schema1.table1", "schema1.table2"]. - - Required: Yes - - Field type: array - - Default value: none - -
- - -- **splitUpdate** - - Description: When the data update type is update, whether to split the update into two pieces of data, see [VI. Data Structure Description]. - - Required: No - - Field type: boolean - - Default value: false - -
- -- **cat** - - Description: The type of operation to be listened to, UPDATE, INSERT, DELETE are available, case insensitive, multiple, split by - - Required: No - - Field type: String - - Default value: UPDATE,INSERT,DELETE - -
- -- **lsn** - - Description: The start position of the SqlServer CDC log sequence number to read - - Required: No - - Field Type: String(00000032:0000002038:0005) - - Default value: None - -
- -- **pollInterval** - - Description: Listen to the interval of pulling SqlServer CDC database, the smaller the value, the smaller the collection delay time, the more pressure on the database access - - Required: No - - Field type: long (in milliseconds) - - Default value: 1000 - -
- - -- **pavingData** - - Description: Whether to paving the parsed json data, see [VII. Description of data structure]. - - Required: No - - Field type: boolean - - Default value: false - -
- -### 2、SQL - -- **url** - - Description: JDBC URL link for sqlserver database - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **username** - - Description: username - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **password** - - Description: Password - - Required: yes - - Parameter type: string - - Default value: none - -
- -- **table** - - Description: The data table to be parsed. - - Note: SQL task only supports listening to a single table and the data format is schema.table - - Required: No - - Field type: string - - Default value: none - -
- -- **cat** - - Description: The type of operation to be listened to, UPDATE, INSERT, DELETE, case insensitive, multiple, split by - - Required: No - - Field type: String - - Default value: UPDATE,INSERT,DELETE - -
- -- **lsn** - - Description: The start position of the SqlServer CDC log sequence number to read - - Required: No - - Field Type: String(00000032:0000002038:0005) - - Default value: None - -
- -- **poll-interval** - - Description: Listen to the interval of pulling SqlServer CDC database, the smaller the value, the smaller the collection delay time, the more pressure on the database access - - Required: No - - Field type: long (in milliseconds) - - Default value: 1000 - -
- -## VII. Data Structure - -On 2020-01-01 12:30:00 (timestamp: 1577853000000) execute. - -```sql -INSERT INTO `tudou`. `kudu`(`id`, `user_id`, `name`) VALUES (1, 1, 'a'); -``` - -On 2020-01-01 12:31:00 (timestamp: 1577853060000) execute. - -```sql -DELETE FROM `tudou`. `kudu` WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; -``` - -On 2020-01-01 12:32:00 (timestamp: 1577853180000) execute. - -```sql -UPDATE `tudou`. `kudu` SET `id` = 2, `user_id` = 2, `name` = 'b' WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; -``` - -1. pavingData = true, splitUpdate = false -The data in RowData are, in order - -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", 1, 1, "a", 2, 2, "b" ] -``` - -2. pavingData = false, splitUpdate = false -The data in RowData are, in order - -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name": "a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name": "a"}, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", {"id":1, "user_id":1, "name": "a"}, {"id":2, "user_id":2, "name": "b"}] -``` - -3. pavingData = true, splitUpdate = true -The data in RowData are, in order - -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null, null] - -//schema, table, ts, opTime, type, before_id, before_user_id, before_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", 1, 1, "a"] - -//schema, table, ts, opTime, type, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", 2, 2, "b"] -``` - -4. pavingData = false, splitUpdate = true -The data in RowData are, in order - -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name": "a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name": "a"}, null] -//schema, table, ts, opTime, type, before -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", {"id":1, "user_id":1, "name": "a"}] -//schema, table, ts, opTime, type, after -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", {"id":2, "user_id":2, "name": "b"}] -``` - -- type: change type, INSERT, UPDATE, DELETE -- opTime: the execution time of SQL in the database -- ts: self-incrementing ID, not repeated, can be used for sorting, decoded as the event time of ChunJun, decoding rules are as follows: - -```java -long id = Long.parseLong("6760525407742726144"); -long res = id >> 22; -DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -System.out.println(sdf.format(res)); //2021-01-28 19:54:21 -``` - -## Eight, data types - -| support | BIT | -| ------------- | ------------------------------------------------------------ | -| | TINYINT24, INT, INTEGER, FLOAT, DOUBLE, REAL, LONG, BIGINT, DECIMAL, NUMERIC | -| | CHAR, VARCHAR, NCHAR, NVARCHAR, TEXT | -| | DATE, TIME, TIMESTAMP, DATETIME, DATETIME2, SMALLDATETIME | -| | BINARY, VARBINARY | -| Not supported | ROWVERSION, UNIQUEIDENTIFIER, CURSOR, TABLE, SQL_VARIANT | - - -
- - -## IX. Sample Scripts - -See the `chunjun-examples` folder in the project. diff --git a/docs/connectors/cassandra/cassandra-lookup.md b/docs/connectors/cassandra/cassandra-lookup.md deleted file mode 100644 index cbf8c4dacb..0000000000 --- a/docs/connectors/cassandra/cassandra-lookup.md +++ /dev/null @@ -1,162 +0,0 @@ -# Cassandra Lookup - -## 一、介绍 - -Cassandra维表,支持全量和异步方式
-全量缓存:将维表数据全部加载到内存中,建议数据量不大,且数据不经常变动的场景使用。
-异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 - -## 二、支持版本 - -Cassandra 常用版本 - -## 三、插件名称 - -| SQL | Cassandra-x | -| --- | --- | - -## 四、参数说明 - -- **host** - - 描述:Cassandra的IP地址,多个地址之间用逗号隔开 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **port** - - 描述:Cassandra连接端口 - - 必选:否 - - 参数类型:int - - 默认值:9042 -
- -- **table-name** - - 描述:要读取Cassandra表名 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **keyspaces** - - 描述:Cassandra keyspaces - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **clusterName** - - 描述:Cassandra cluster name - - 必选:否 - - 参数类型:string - - 默认值:chunjun-cluster -
- -- **consistency** - - 描述:Cassandra consistence(一致性) - - 说明:一致性级别决定了副本中必须有多少节点响应协调器节点才能成功处理非轻量级事务。 - - 必选:否 - - 参数类型:string - - 默认值:LOCAL_QUORUM -
- -- **coreConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可用连接数 - - 必选:否 - - 参数类型:int - - 默认值:8 -
- -- **maxConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可连接数 - - 必选:否 - - 参数类型:int - - 默认值:32768 -
- -- **maxRequestsPerConnection** - - 描述:Cassandra 每个连接的最多请求数 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **maxQueueSize** - - 描述:Cassandra 队列最大数 - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **readTimeoutMillis** - - 描述:Cassandra read 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **poolTimeoutMillis** - - 描述:Cassandra pool 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **connectTimeoutMillis** - - 描述:Cassandra connect 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **lookup.cache-type** - - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - - 必选:否 - - 参数类型:string - - 默认值:LRU -
- -- **lookup.cache-period** - - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - - 必选:否 - - 参数类型:string - - 默认值:3600000 -
- -- **lookup.cache.max-rows** - - 描述:lru维表缓存数据的条数,默认10000条 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **lookup.cache.ttl** - - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - - 必选:否 - - 参数类型:string - - 默认值:60000 -
- -- **lookup.fetch-size** - - 描述:ALL维表每次从数据库加载的条数,默认1000条 - - 必选:否 - - 参数类型:string - - 默认值:1000 -
- -- **lookup.parallelism** - - 描述:维表并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -## 五、数据类型 - -| 支持 | BYTE、INT、FLOAT、DOUBLE、BOOLEAN、TEXT、VARCHAR、DECIMAL、TIMESTAMP | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - -## 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/cassandra/cassandra-sink.md b/docs/connectors/cassandra/cassandra-sink.md deleted file mode 100644 index 62a93285d6..0000000000 --- a/docs/connectors/cassandra/cassandra-sink.md +++ /dev/null @@ -1,222 +0,0 @@ -# Cassandra Sink - -## 一、介绍 - -Cassandra sink - -## 二、支持版本 - -主流版本 - -## 三、插件名称 - -| Sync | Cassandrasink、Cassandrawriter | -| --- | --- | -| SQL | Cassandra-x | - -## 四、参数说明 - -### 1、Sync - -- **host** - - 描述:Cassandra的IP地址,多个地址之间用逗号隔开 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **port** - - 描述:Cassandra连接端口 - - 必选:否 - - 参数类型:int - - 默认值:9042 -
- -- **table-name** - - 描述:要读取Cassandra表名 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **keyspaces** - - 描述:Cassandra keyspaces - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **clusterName** - - 描述:Cassandra cluster name - - 必选:否 - - 参数类型:string - - 默认值:chunjun-cluster -
- -- **consistency** - - 描述:Cassandra consistence(一致性) - - 说明:一致性级别决定了副本中必须有多少节点响应协调器节点才能成功处理非轻量级事务。 - - 必选:否 - - 参数类型:string - - 默认值:LOCAL_QUORUM -
- -- **coreConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可用连接数 - - 必选:否 - - 参数类型:int - - 默认值:8 -
- -- **maxConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可连接数 - - 必选:否 - - 参数类型:int - - 默认值:32768 -
- -- **maxRequestsPerConnection** - - 描述:Cassandra 每个连接的最多请求数 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **maxQueueSize** - - 描述:Cassandra 队列最大数 - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **readTimeoutMillis** - - 描述:Cassandra read 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **poolTimeoutMillis** - - 描述:Cassandra pool 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **connectTimeoutMillis** - - 描述:Cassandra connect 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -### 2、SQL - -- **host** - - 描述:Cassandra的IP地址,多个地址之间用逗号隔开 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **port** - - 描述:Cassandra连接端口 - - 必选:否 - - 参数类型:int - - 默认值:9042 -
- -- **table-name** - - 描述:要读取Cassandra表名 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **keyspaces** - - 描述:Cassandra keyspaces - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **clusterName** - - 描述:Cassandra cluster name - - 必选:否 - - 参数类型:string - - 默认值:chunjun-cluster -
- -- **consistency** - - 描述:Cassandra consistence(一致性) - - 说明:一致性级别决定了副本中必须有多少节点响应协调器节点才能成功处理非轻量级事务。 - - 必选:否 - - 参数类型:string - - 默认值:LOCAL_QUORUM -
- -- **coreConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可用连接数 - - 必选:否 - - 参数类型:int - - 默认值:8 -
- -- **maxConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可连接数 - - 必选:否 - - 参数类型:int - - 默认值:32768 -
- -- **maxRequestsPerConnection** - - 描述:Cassandra 每个连接的最多请求数 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **maxQueueSize** - - 描述:Cassandra 队列最大数 - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **readTimeoutMillis** - - 描述:Cassandra read 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **poolTimeoutMillis** - - 描述:Cassandra pool 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **connectTimeoutMillis** - - 描述:Cassandra connect 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **sink.parallelism** - - 描述:sink并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -## 五、数据类型 - -| 支持 | BYTE、INT、FLOAT、DOUBLE、BOOLEAN、TEXT、VARCHAR、DECIMAL、TIMESTAMP | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION - -## 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/cassandra/cassandra-source.md b/docs/connectors/cassandra/cassandra-source.md deleted file mode 100644 index 489e3cfa9e..0000000000 --- a/docs/connectors/cassandra/cassandra-source.md +++ /dev/null @@ -1,224 +0,0 @@ -## 一、介绍 - -读取Cassandra数据 - -## 二、支持版本 - -Cassandra主流版本 - -## 三、插件名称 - -| Sync | Cassandrasource、Cassandrareader | -| --- | --- | -| SQL | Cassandra-x | - -## 四、参数说明 - -#### 1、数据同步 - -- **host** - - 描述:Cassandra的IP地址,多个地址之间用逗号隔开 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **port** - - 描述:Cassandra连接端口 - - 必选:否 - - 参数类型:int - - 默认值:9042 -
- -- **table-name** - - 描述:要读取Cassandra表名 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **keyspaces** - - 描述:Cassandra keyspaces - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **clusterName** - - 描述:Cassandra cluster name - - 必选:否 - - 参数类型:string - - 默认值:chunjun-cluster -
- -- **consistency** - - 描述:Cassandra consistence(一致性) - - 说明:一致性级别决定了副本中必须有多少节点响应协调器节点才能成功处理非轻量级事务。 - - 必选:否 - - 参数类型:string - - 默认值:LOCAL_QUORUM -
- -- **coreConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可用连接数 - - 必选:否 - - 参数类型:int - - 默认值:8 -
- -- **maxConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可连接数 - - 必选:否 - - 参数类型:int - - 默认值:32768 -
- -- **maxRequestsPerConnection** - - 描述:Cassandra 每个连接的最多请求数 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **maxQueueSize** - - 描述:Cassandra 队列最大数 - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **readTimeoutMillis** - - 描述:Cassandra read 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **poolTimeoutMillis** - - 描述:Cassandra pool 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **connectTimeoutMillis** - - 描述:Cassandra connect 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **column** - - 描述:需要读取的字段。 - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - 必选:是 - - 字段类型:List - - 默认值: - -#### 2、SQL - -- **host** - - 描述:Cassandra的IP地址,多个地址之间用逗号隔开 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **port** - - 描述:Cassandra连接端口 - - 必选:否 - - 参数类型:int - - 默认值:9042 -
- -- **table-name** - - 描述:要读取Cassandra表名 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **keyspaces** - - 描述:Cassandra keyspaces - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **clusterName** - - 描述:Cassandra cluster name - - 必选:否 - - 参数类型:string - - 默认值:chunjun-cluster -
- -- **consistency** - - 描述:Cassandra consistence(一致性) - - 说明:一致性级别决定了副本中必须有多少节点响应协调器节点才能成功处理非轻量级事务。 - - 必选:否 - - 参数类型:string - - 默认值:LOCAL_QUORUM -
- -- **coreConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可用连接数 - - 必选:否 - - 参数类型:int - - 默认值:8 -
- -- **maxConnectionsPerHost** - - 描述:Cassandra 每个地址可供最多可连接数 - - 必选:否 - - 参数类型:int - - 默认值:32768 -
- -- **maxRequestsPerConnection** - - 描述:Cassandra 每个连接的最多请求数 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **maxQueueSize** - - 描述:Cassandra 队列最大数 - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **readTimeoutMillis** - - 描述:Cassandra read 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **poolTimeoutMillis** - - 描述:Cassandra pool 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- -- **connectTimeoutMillis** - - 描述:Cassandra connect 超时时长 - - 必选:否 - - 参数类型:int - - 默认值:60 * 1000 -
- - -## 五、数据类型 - -| 支持 | BYTE、INT、FLOAT、DOUBLE、BOOLEAN、TEXT、VARCHAR、DECIMAL、TIMESTAMP | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION - - -## 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/connectors-en/logminer/LogMiner-source.md b/docs/connectors/connectors-en/logminer/LogMiner-source.md deleted file mode 100644 index 89deabf071..0000000000 --- a/docs/connectors/connectors-en/logminer/LogMiner-source.md +++ /dev/null @@ -1,388 +0,0 @@ -# LogMiner Source - - - -- [LogMiner Source](#logminer-source) - - [I、Introduction](#I Introduction) - - [II、Supported Versions](#II Supported Versions) - - [III、Database Configuration](#III Database Configuration) - - [IV、LogMinerPrinciple](#IV logminer principle) - - [V、Plugin Name](#V Plugin Name) - - [VI、Parameter Description](#VI Parameter Description) - - [i、Sync](#isync) - - [ii、SQL](#iisql) - - [VII、Data Structure](#VII Data Structure) - - [VIII、Data Types](#VIII Data Types) - - [Ⅸ、Script Example](#ⅨScript Example) - - - -
- -## I、Introduction -The oraclelogminer plug-in supports configuring the name of the listening table and the starting point for reading log data. Oraclelogminer saves the current consumption points during checkpoint, so it supports continuation.
- -## II、Supported Versions -oracle10,oracle11,oracle12,oracle19,RAC,master-slave -
- -## III、Database Configuration -[OracleConfigurationLogMiner](LogMinerConfiguration.md) -
- -## IV、LogMinerPrinciple -[LogMinerPrinciple](LogMinerPrinciple.md) -
- -## V、Plugin Name -| Sync | oraclelogminerreader、oraclelogminersource | -| --- | --- | -| SQL | oraclelogminer-x | - -## -## VI、Parameter Description -### i、Sync - -- **jdbcUrl** - - Description:jdbc url of Oracle database, - - Required:Yes - - Field type:string - - Default value:none - -
- - -- **username** - - Description:The username of the data source - - Required:Yes - - Field type:string - - Default:none - -
- -- **password** - - Description:The password of the username specified by the data source - - Required:Yes - - Field type:string - - Default:none - -
- -- **table** - - Description: The format of the table to be monitored is: schema.table. Schema cannot be configured as *, but table can be configured to * monitor all tables under the specified library, such as: [schema1. Table1 "," schema1. Table2 "," schema2. *] - - Required:No - - Field type:array - - Default:none - -
- - -- **splitUpdate** - - Description:When the data update type is update, whether to split the update into two data, see [七、Data Structure](#七Data Structure) - - Required: No - - Field type: boolean - - Default value: false - -
- -- **cat** - - Description:Type of operation data monitored,including insert, update, and delete - - Required:No - - Field type:String - - Default value:UPDATE,INSERT,DELETE - -
- -- **readPosition** - - Description:Starting point of Oracle Logminer - - Optional value: - - all: Start collection from the oldest archive log group in the Oracle database (not recommended) - - current:Start collection from task runtime - - time: Start collection from a specified point in time - - scn: Start acquisition from the specified SCN number - - Required:No - - Field type:String - - Default value:current - -
- - -- **startTime** - - Description: Specifies the millisecond timestamp of the collection start point - - Required:This parameter is required when 'readposition' is' time ' - - Field type:Long(Millisecond timestamp) - - Default value:none - -
- -- **startSCN** - - Description: Specifies the SCN number of the collection start point - - Required:This parameter is required when 'readposition' is' SCN ' - - Field type:String - - Default value:none - -
- -- **fetchSize** - - Description: Batch from v$logmnr_contents The number of data pieces pulled in the contents view. For data changes with a large amount of data, increasing this value can increase the reading speed of the task to a certain extent - - Required:No - - Field type:Integer - - Default value:1000 - -
- -- **queryTimeout** - - Description: Timeout parameter of logminer executing SQL query, unit: seconds - - Required:No - - Field typ:Long - - Default value:300 - -
- -- **supportAutoAddLog** - - Description:Whether to automatically add log groups when starting logminer (not recommended) - - Required:No - - Field typ:Boolean - - Default value:false - -
- -- **pavingData** - - Description:Whether to flatten the parsed JSON data, see [七、Data Structure](#七Data Structure) - - Required:No - - Field typ:boolean - - Default value:false - -
- -### ii、SQL -- **url** - - Description:jdbc url of Oracle database, - - Required:Yes - - Field type:string - - Default value:none - -
- -- **username** - - Description:The username of the data source - - Required:Yes - - Field type:string - - Default:none - -
- -- **password** - - Description:The password of the username specified by the data source - - Required:Yes - - Field type:string - - Default:none - -
- -- **table** - - Description: The format of the table to be monitored is: schema.table. - - Note:SQL tasks only support listening to a single table - - Required:No - - Field type:string - - Default:none - -
- -- **cat** - - Description:Type of operation data monitored,including insert, update, and delete - - Required:No - - Field type:String - - Default value:UPDATE,INSERT,DELETE - -
- - - -- **read-position** - - Description:Starting point of Oracle Logminer - - Optional value: - - all: Start collection from the oldest archive log group in the Oracle database (not recommended) - - current:Start collection from task runtime - - time: Start collection from a specified point in time - - scn: Start acquisition from the specified SCN number - - Required:No - - Field type:String - - Default value:current - -
- - -- **start-time** - - Description: Specifies the millisecond timestamp of the collection start point - - Required:This parameter is required when 'read-position' is' time ' - - Field type:Long(Millisecond timestamp) - - Default value:none - -
- -- **start-scn** - - Description: Specifies the SCN number of the collection start point - - Required:This parameter is required when 'read-position' is' SCN ' - - Field type:String - - Default value:none - -
- -- **fetch-size** - - Description: Batch from v$logmnr_contents The number of data pieces pulled in the contents view. For data changes with a large amount of data, increasing this value can increase the reading speed of the task to a certain extent - - Required:No - - Field type:Integer - - Default value:1000 - -
- -- **query-timeout** - - Description: Timeout parameter of logminer executing SQL query, unit: seconds - - Required:No - - Field type:Long - - Default value:300 - -
- - -- **support-auto-add-log** - - Description:Whether to automatically add log groups when starting logminer (not recommended) - - Required:No - - Field type:Boolean - - Default value:false - -
- -- **io-threads** - - Description:The maximum number of IO processing threads is three - - Required:No - - Field type:int - - Default value:1 - -
- -- **max-log-file-size** - - Description:the size of the log file loaded at one time. The default is 5g. The unit is byte - - Required:No - - Field type:long - - Default value:5*1024*1024*1024 - -
- -- **transaction-cache-num-size** - - Description:The number of DML SQL that logminer cache - - Required:No - - Field type:long - - Default value:800 - -
- -- **transaction-expire-time** - - Description:The expiration time of logMiner cache, unit minutes - - Required:No - - Field type:int - - Default value:20 - -
-## VII、Data Structure - -execute sql at 2021-06-29 23:42:19(timeStamp:1624981339000): -```sql -INSERT INTO TIEZHU.RESULT1 ("id", "name", "age") VALUES (1, 'a', 12) -``` -
- -execute sql at 2021-06-29 23:42:29(timeStamp:1624981349000): -```sql -UPDATE TIEZHU.RESULT1 t SET t."id" = 2, t."age" = 112 WHERE t."id" = 1 -``` -
- -execute sql at 2021-06-29 23:42:34(timeStamp:1624981354000): -```sql - DELETE FROM TIEZHU.RESULT1 WHERE "id" = 2 -``` -
- -1、pavingData = true, splitUpdate = false -The data in rowdata is: -``` -//scn schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a", 12] -[49982969,"TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", 1, "a", 12 , 2, "a", 112] -[49982973,"TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a",112 , null, null, null] -``` -
- -2、pavingData = false, splitUpdate = false -The data in rowdata is: -``` -//scn, schema, table, ts, opTime, type, before, after -[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", {"id":1, "name":"a", "age":12}, {"id":2, "name":"a", "age":112}] -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] -``` -
- -3、pavingData = true, splitUpdate = true -The data in rowdata is: -``` -//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a",12 ] - -//scn, schema, table, opTime, ts, type, before_id, before_name, before_age -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", 1, "a", 12] -//scn, schema, table, opTime, ts, type, after_id, after_name, after_age -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", 2, "a", 112] - -//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a", 112, null, null, null] - - -``` -
- -4、pavingData = false, splitUpdate = true -The data in rowdata is: -``` -//scn, schema, table, ts, opTime, type, before, after -[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] -//scn, schema, table, ts, opTime, type, before -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", {"id":1, "name":"a", "age":12}] -//scn, schema, table, ts, opTime, type, after -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", {"id":2, "name":"a", "age":112}] -//scn, schema, table, ts, opTime, type, before, after -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] - -``` -
- -- scn:SCN number corresponding to Oracle database change record -- type:DML Type,INSERT,UPDATE、DELETE -- opTime:Execution time of SQL in database -- ts:Self incrementing ID, non duplicate, can be used for sorting. After decoding, it is the event time of chunjun. The decoding rules are as follows: -
- -```java -long id = Long.parseLong("6815665753853923328"); - long res = id >> 22; - DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - System.out.println(sdf.format(res)); //2021-06-29 23:42:24 -``` - -
- -## VIII、Data Types -| Support | NUMBER、SMALLINT、INT INTEGER、FLOAT、DECIMAL、NUMERIC、BINARY_FLOAT、BINARY_DOUBLE | -| -- | -- | -| | CHAR、NCHAR、NVARCHAR2、ROWID、VARCHAR2、VARCHAR、LONG、RAW、LONG RAW、INTERVAL YEAR、INTERVAL DAY、BLOB、CLOB、NCLOB | -| | DATE、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZONE | -| Not Support | BFILE、XMLTYPE、Collections | - - -
- - -## Ⅸ、Script Example -See the `chunjun-examples` folder in the project。 diff --git a/docs/connectors/doris/dorisbatch-sink.md b/docs/connectors/doris/dorisbatch-sink.md deleted file mode 100644 index b85d2be940..0000000000 --- a/docs/connectors/doris/dorisbatch-sink.md +++ /dev/null @@ -1,126 +0,0 @@ -# Doris batch Sink - -## 一、介绍 -Doris batch Sink插件支持向Doris数据库写入数据 - -## 二、支持版本 -Doris 0.14.x - -## 三、插件名称 -| Sync | dorisbatchsink、dorisbatchwriter | -| --- | --- | -| SQL | | - -## 四、插件参数 - - -### 1.Sync - -- **feNodes** - - 描述:连接Doris的Fe Nodes 地址 - - 必选:是 - - 字段类型:List - - 示例: -```json -"feNodes": ["127.0.0.1:8030"] -``` - - - 默认值:无 - -
- - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 字段类型:List - - 默认值:无 - - -
- -- **database** - - 描述:写入Doris的库名 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **column** - - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - - 必选:是 - - 字段类型:List - - 默认值:无 - - -
- -- **fieldDelimiter** - - 描述:写入Doris数值的字段分隔符 - - 必选:否 - - 字段类型:String - - 默认值:\t - - -
- -- **lineDelimiter** - - 描述:写入Doris数值的行分隔符 - - 必选:否 - - 字段类型:String - - 默认值:\n - - -
- -- **loadProperties** - - 描述:针对Doris写入任务的系统参数,主要是针对Doris的特定配置 - - 必选:否 - - 字段类型:Object - - 默认值:无 - - -
- -- **batchSize** - - 描述:批量写入Doris的数据量大小 - - 必选:否 - - 字段类型:int - - 默认值:1000 - - -
- - - -### 2.SQL - - - -## 五、数据类型 -|支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、LARGEINT、FLOAT、DOUBLE、DECIMAL、DATE、DATETIME、CHAR、VARCHAR、STRING| -| --- | --- | - - -## 六、脚本示例 - 见项目内`**chunjun-examples**`文件夹。 diff --git a/docs/connectors/elasticsearch5/es5-sink.md b/docs/connectors/elasticsearch5/es5-sink.md deleted file mode 100644 index 15362e662b..0000000000 --- a/docs/connectors/elasticsearch5/es5-sink.md +++ /dev/null @@ -1,130 +0,0 @@ -# 一、介绍 -ElasticSearch Sink插件支持将数据写入到指定的index中去。 -​ - -# 二、支持版本 -Elasticsearch 5.x -​ - -# 三、插件名称 - - -| 类型|名称| -| ---- | ----| -| Sync | elasticsearch5writer | -| SQL | elasticsearch5-x | - -​ - -# 四、参数说明 - - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9300"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- cluster - - 描述:指定访问Elasticsearch的集群名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- batchSize - - 描述:批量写入数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- keyDelimiter - - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - - 必须:否 - - 参数类型:无 - - 默认值:"_" -- column - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: - -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ - -## 2、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9300"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- document-type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- cluster - - 描述:指定访问Elasticsearch的集群名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- bulk-flush.max-actions - - 描述:一次性读取es数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- document-id.key-delimiter - - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - - 必须:否 - - 参数类型:String - - 默认值:"_" - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 -见项目内`ChunJun:Local:Test`模块中的`demo文件夹。 - diff --git a/docs/connectors/elasticsearch5/es5-source.md b/docs/connectors/elasticsearch5/es5-source.md deleted file mode 100644 index 7b1c1fb0e1..0000000000 --- a/docs/connectors/elasticsearch5/es5-source.md +++ /dev/null @@ -1,120 +0,0 @@ -# 一、介绍 -ElasticSearch Source插件支持从现有的ElasticSearch集群读取指定index中的数据。 -​ - -# 二、支持版本 -Elasticsearch 5.x -​ - -# 三、插件名称 - -| 类型|名称| -| --- | --- | -| Sync | elasticsearch5reader | -| SQL | elasticsearch5-x | - -​ - -# 四、参数说明 - - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9300"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- cluster - - 描述:指定访问Elasticsearch的集群名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- batchSize - - 描述:批量读取数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- column - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: - -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ - -## 2、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9300"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- document-type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- cluster - - 描述:指定访问Elasticsearch的集群名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- bulk-flush.max-actions - - 描述:一次性读取es数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 - -​ - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 -见项目内ChunJun:Local:Test模块中的demo文件夹。 diff --git a/docs/connectors/elasticsearch6/es6-lookup.md b/docs/connectors/elasticsearch6/es6-lookup.md deleted file mode 100644 index e6108ce775..0000000000 --- a/docs/connectors/elasticsearch6/es6-lookup.md +++ /dev/null @@ -1,86 +0,0 @@ -# 一、介绍 -ElasticSearch Lookup插件支持从现有的ElasticSearch集群读取指定index中的数据,并作为维表与主表进行关联。目前维表支持全量维表和异步维表。 - -# 二、支持版本 - -Elasticsearch 6.x - -# 三、插件名称 - - -|类型|名称| -| --- | --- | -| SQL | es-x | - - -​
-# 四、参数说明 -## 1、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: 'http://host_name:9092;http://host_name:9093' - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- lookup.cache-type - - 描述:维表类型。eg: all 或者 lru - - 必须:否 - - 参数类型:String - - 默认值:LRU -- lookup.cache-period - - 描述:全量维表周期时间 - - 必须:否 - - 参数类型:Long - - 默认值:3600 * 1000L -- lookup.cache.max-rows - - 描述:维表缓存的最大条数 - - 必须:否 - - 参数类型:Long - - 默认值:1000L -- lookup.cache.ttl - - 描述:缓存生命周期 - - 必须:否 - - 参数类型:Long - - 默认值:60 * 1000L -- lookup.error-limit - - 描述:维表数据不合规条数 - - 必须:否 - - 参数类型:Long - - 默认值:Long.MAX_VALUE -- lookup.fetch-size - - 描述:抓取维表数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1000L -- lookup.parallelism - - 描述:维表并行度 - - 必须:否 - - 参数类型:Integer - - 默认值:1 - - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/elasticsearch6/es6-sink.md b/docs/connectors/elasticsearch6/es6-sink.md deleted file mode 100644 index 57ee94079e..0000000000 --- a/docs/connectors/elasticsearch6/es6-sink.md +++ /dev/null @@ -1,120 +0,0 @@ -# 一、介绍 -ElasticSearch Sink插件支持将数据写入到指定的index中。 -​ - -# 二、支持版本 -Elasticsearch 6.x -​ - -# 三、插件名称 - - -| 类型|名称| -| ---- | ----| -| Sync | eswriter、essink | -| SQL | es-x | - -​ - -# 四、参数说明 - - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9200"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- batchSize - - 描述:批量写入数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- keyDelimiter - - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - - 必须:否 - - 参数类型:无 - - 默认值:"_" -- column - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: - -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ - -## 2、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: 'http://host_name:9092;http://host_name:9093' - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- document-type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- bulk-flush.max-actions - - 描述:一次性读取es数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1000 -- document-id.key-delimiter - - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - - 必须:否 - - 参数类型:String - - 默认值:"_" - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/elasticsearch6/es6-source.md b/docs/connectors/elasticsearch6/es6-source.md deleted file mode 100644 index 27d238d930..0000000000 --- a/docs/connectors/elasticsearch6/es6-source.md +++ /dev/null @@ -1,108 +0,0 @@ -# 一、介绍 -ElasticSearch Source插件支持从现有的ElasticSearch集群读取指定index中的数据。 -​ - -# 二、支持版本 -Elasticsearch 6.x -​ - -# 三、插件名称 - -| 类型|名称| -| --- | --- | -| Sync | esreader、essource | -| SQL | es-x | - -​ - -# 四、参数说明 - - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9200"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- type - - 描述:指定访问Elasticsearch集群的index下的type名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- batchSize - - 描述:批量读取数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- column - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: - -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ - -## 2、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: 'http://host_name:9092;http://host_name:9093' - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- document-type - -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- bulk-flush.max-actions - - 描述:一次性读取es数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1000 - -​ - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 - -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/elasticsearch7/es7-source.md b/docs/connectors/elasticsearch7/es7-source.md deleted file mode 100644 index 319652eaf6..0000000000 --- a/docs/connectors/elasticsearch7/es7-source.md +++ /dev/null @@ -1,149 +0,0 @@ -# 一、介绍 -ElasticSearch Source插件支持从现有的ElasticSearch集群读取指定index中的数据。 -​ - -# 二、支持版本 -Elasticsearch 7.x -​ - -# 三、插件名称 - -| 类型|名称| -| --- | --- | -| Sync | elasticsearch7reader | -| SQL | elasticsearch7-x | - -​ - -# 四、参数说明 - - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9200"] - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- batchSize - - 描述:批量读取数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- column - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: -- connectTimeout - - 描述:ES Client最大连接超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:5000 -- socketTimeout - - 描述:ES Client最大socket超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:1800000 -- keepAliveTime - - 描述:ES Client会话最大保持时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:5000 -- requestTimeout - - 描述:ES Client最大请求超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:2000 -- maxConnPerRoute - - 描述:每一个路由值的最大连接数量 - - 必须:否 - - 参数类型:Integer - - 默认值:10 -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ - -## 2、SQL - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: "localhost:9200",多个地址用分号作为分隔符。 - - 必选:是 - - 参数类型:List - - 默认值:无 -- index - - 描述:指定访问Elasticsearch集群的index名称 - - 必选:是 - - 参数类型:String - - 默认值:无 -- username - - 描述:开启basic认证之后的用户名 - - 必须:否 - - 参数类型:String - - 默认值:无 -- password - - 描述:开启basic认证之后的密码 - - 必须:否 - - 参数类型:String - - 默认值:无 -- bulk-flush.max-actions - - 描述:一次性读取es数据的条数 - - 必须:否 - - 参数类型:Integer - - 默认值:1 -- client.connect-timeout - - 描述:ES Client最大连接超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:5000 -- client.socket-timeout - - 描述:ES Client最大socket超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:1800000 -- client.keep-alive-time - - 描述:ES Client会话最大保持时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:5000 -- client.request-timeout - - 描述:ES Client最大请求超时时间。 - - 必须:否 - - 参数类型:Integer - - 默认值:2000 -- client.max-connection-per-route - - 描述:每一个路由值的最大连接数量 - - 必须:否 - - 参数类型:Integer - - 默认值:10 - -​ - -# 五、数据类型 - -|是否支持 | 类型名称 | -| --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| - -# 六、脚本示例 -见项目内ChunJun:Local:Test模块中的demo文件夹。 diff --git a/docs/connectors/emqx/emqx-sink.md b/docs/connectors/emqx/emqx-sink.md deleted file mode 100644 index ff36b599a3..0000000000 --- a/docs/connectors/emqx/emqx-sink.md +++ /dev/null @@ -1,127 +0,0 @@ -# Emqx Sink - -## 一、介绍 -emqx sink - -## 二、支持版本 -主流版本 - - -## 三、插件名称 -| Sync | emqxsink、emqxwriter | -| --- | --- | -| SQL | emqx-x | - - -## 四、参数说明 -### 1、Sync -- **broker** - - 描述:连接URL信息 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **topic** - - 描述:订阅主题 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **username** - - 描述:认证用户名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **password** - - 描述:认证密码 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **isCleanSession** - - 描述:是否清除session - - false:MQTT服务器保存于客户端会话的的主题与确认位置; - - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - 必选:否 - - 参数类型:boolean - - 默认值:true -
- -- **qos** - - 描述:服务质量 - - 0:AT_MOST_ONCE,至多一次; - - 1:AT_LEAST_ONCE,至少一次; - - 2:EXACTLY_ONCE,精准一次; - - 必选:否 - - 参数类型:int - - 默认值:2 -
- -### 2、SQL -- **connector** - - 描述:emqx-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **broker** - - 描述:连接信息tcp://localhost:1883 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **topic** - - 描述:主题 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **isCleanSession** - - 描述:是否清除session - - false:MQTT服务器保存于客户端会话的的主题与确认位置; - - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - 必选:否 - - 参数类型:String - - 默认值:true -
- -- **qos** - - 描述:服务质量 - - 0:AT_MOST_ONCE,至多一次; - - 1:AT_LEAST_ONCE,至少一次; - - 2:EXACTLY_ONCE,精准一次; - - 必选:否 - - 参数类型:String - - 默认值:2 -
- -- **username** - - 描述:username - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| 暂不支持 | 其他 | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/emqx/emqx-source.md b/docs/connectors/emqx/emqx-source.md deleted file mode 100644 index 736bc4cf75..0000000000 --- a/docs/connectors/emqx/emqx-source.md +++ /dev/null @@ -1,146 +0,0 @@ -# Emqx Source - -## 一、介绍 -从emq实时读取数据 - -## 二、支持版本 -主流版本 - - -## 三、插件名称 -| Sync | emqxsource、emqxreader | -| --- | --- | -| SQL | emqx-x | - -## 四、参数说明 -### 1、Sync -- **broker** - - 描述:连接URL信息 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **topic** - - 描述:订阅主题 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **username** - - 描述:认证用户名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **password** - - 描述:认证密码 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **isCleanSession** - - 描述:是否清除session - - false:MQTT服务器保存于客户端会话的的主题与确认位置; - - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - 必选:否 - - 参数类型:boolean - - 默认值:true -
- -- **qos** - - 描述:服务质量 - - 0:AT_MOST_ONCE,至多一次; - - 1:AT_LEAST_ONCE,至少一次; - - 2:EXACTLY_ONCE,精准一次; - - 必选:否 - - 参数类型:int - - 默认值:2 -
- -- **codec** - - 描述:编码解码器类型,支持 json、plain - - plain:将kafka获取到的消息字符串存储到一个key为message的map中,如:`{"message":"{\"key\": \"key\", \"message\": \"value\"}"}` - - plain:将kafka获取到的消息字符串按照json格式进行解析 - - 若该字符串为json格式 - - 当其中含有message字段时,原样输出,如:`{"key": "key", "message": "value"}` - - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,如:`{"key": "key", "value": "value", "message": "{\"key\": \"key\", \"value\": \"value\"}"}` - - 若改字符串不为json格式,则按照plain类型进行处理 - - 必选:否 - - 参数类型:string - - 默认值:plain -
- -### 2、SQL -- **connector** - - 描述:emqx-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **broker** - - 描述:连接信息tcp://localhost:1883 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **topic** - - 描述:主题 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **isCleanSession** - - 描述:是否清除session - - false:MQTT服务器保存于客户端会话的的主题与确认位置; - - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - 必选:否 - - 参数类型:String - - 默认值:true -
- -- **qos** - - 描述:服务质量 - - 0:AT_MOST_ONCE,至多一次; - - 1:AT_LEAST_ONCE,至少一次; - - 2:EXACTLY_ONCE,精准一次; - - 必选:否 - - 参数类型:String - - 默认值:2 -
- -- **username** - - 描述:username - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **format** - - 描述:数据来源格式 - - 必选:否 - - 参数类型:String - - 默认值:json -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| 暂不支持 | 其他 | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/file/file-source.md b/docs/connectors/file/file-source.md deleted file mode 100644 index b874ca8932..0000000000 --- a/docs/connectors/file/file-source.md +++ /dev/null @@ -1,39 +0,0 @@ -# File Source - -## 一、介绍 -file source仅支持从本地路径读取文件,支持读取原生flink所有文件类型。
- -## 二、插件名称 -| SQL | file-x | -| --- | --- | - - -## 三、参数说明 -### 1、SQL - -- **path** - - 描述:读取的数据文件路径 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **format** - - 描述:文件的类型,和原生flink保持一致,支持原生所有类型 - - 必选:否 - - 参数类型:string - - 默认值:csv -
- -- **encoding** - - 描述:字符编码 - - 必选:否 - - 字段类型:string - - 默认值:`UTF-8` - -## 四、数据类型 -和原生flink数据类型保持一致
每种format所支持的数据类型请参考[flink官方文档](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/formats/)
- -## 五、脚本示例 -见项目内`chunjun-examples`文件夹。 - diff --git a/docs/connectors/filesystem/filesystem-sink.md b/docs/connectors/filesystem/filesystem-sink.md deleted file mode 100644 index 51a6bdf6bc..0000000000 --- a/docs/connectors/filesystem/filesystem-sink.md +++ /dev/null @@ -1,106 +0,0 @@ -# FileSystem Sink - -## 一、介绍 -filesystem sink 是文件写入系统的公共插件。 具体的的插件需要具体实现 ,用户只要配置不同的文件路径,系统 -动态加载不同的文件系统插件,如hdfs:// s3a:// 等
- -## 二、插件名称 -| SQL | filesystem-x | -| --- |--------------| - - -## 三、参数说明 -### 1、公共参数 - -- **path** - - 描述:读取的数据文件路径 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **format** - - 描述:文件的类型,和原生flink保持一致,支持原生所有类型,详情见[flink官方文档](https://nightlies.apache.org/flink/flink-docs-release-1.12/zh/dev/table/connectors/filesystem.html#file-formats) - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **sink.rolling-policy.file-size** - - 描述:文件写入系统的大小 - - 必选:无 - - 参数类型:MemorySize - - 默认值:128MB -
- -- **sink.rolling-policy.rollover-interval** - - 描述:文件写入系统的时间间隔 - - 必选:无 - - 参数类型:Duration - - 默认值:30 min -
- -- **sink.rolling-policy.check-interval** - - 描述:检查文件是否需要写入的定时任务启动间隔 - - 必选:无 - - 参数类型:Duration - - 默认值:1 min -
- -- **auto-compaction** - - 描述:单次checkpoint产生的文件是否需要进行合并 - - 必选:无 - - 参数类型:boolean - - 默认值:false -
- -- **compaction.file-size** - - 描述:单次checkpoint产生的文件合并的最大大小 - - 必选:无 - - 参数类型:MemorySize - - 默认值:sink.rolling-policy.file-size -
- -### 2、s3数据源参数 - -- **path** - - 描述:此参数是公共参数,当数据源为s3 时,路径格式必须是s3a://{bucket}/{your path} - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **s3.access-key** - - 描述:s3 SecretId - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **s3.secret-key** - - 描述:s3 SecretKey - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **s3.endpoint** - - 描述:s3.endpoint - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **s3.path.style.access** - - 描述:某些兼容 S3 的对象存储服务可能没有默认启用虚拟主机样式的寻址,此配置以启用路径样式的访问 - - 必选:否 - - 参数类型:boolean - - 默认值:true -
- -## 四、数据类型 -和原生flink数据类型保持一致
每种format所支持的数据类型请参考[flink官方文档](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/formats/)
- -## 五、脚本示例 -见项目内`chunjun-examples`文件夹。 - diff --git a/docs/connectors/ftp/ftp-sink.md b/docs/connectors/ftp/ftp-sink.md deleted file mode 100644 index e9dcdc40f3..0000000000 --- a/docs/connectors/ftp/ftp-sink.md +++ /dev/null @@ -1,233 +0,0 @@ -# Ftp Sink - -## 一、介绍 -ftp sink - -## 二、数据源配置 -FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
- -## 三、插件名称 -| sync | ftpsink, ftpwriter | -| --- | --- | -| sql | ftp-x | - - -## 四、参数说明 - -### 1、sync - -- **path** - - 描述:数据文件路径 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **protocol** - - 描述:服务器访问协议,目前支持ftp、sftp - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **host** - - 描述:ftp服务器地址 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **port** - - 描述:ftp服务器端口 - - 必选:否 - - 字段类型:int - - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 -
- -- **username** - - 描述:ftp服务器登陆用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **password** - - 描述:ftp服务器登陆密码 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **privateKeyPath** - - 描述:sftp私钥文件路径 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **connectPattern** - - 描述:protocol为ftp时的连接模式,可选PASV和PORT,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) - - 必选:否 - - 字段类型:String - - 默认值:PASV -
- -- **fieldDelimiter** - - 描述:读取的字段分隔符 - - 必选:否 - - 字段类型:String - - 默认值:, -
- -- **encoding** - - 描述:读取文件的编码配置 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 -
- -- **controlEncoding** - - 描述:FTP客户端编码格式 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 -
- -- **isFirstLineHeader** - - 描述:首行是否为标题行,如果是则不读取第一行 - - 必选:否 - - 字段类型:boolean - - 默认值:false -
- -- **ftpFileName** - - 描述:文件名 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **timeout** - - 描述:连接超时时间,单位毫秒 - - 必选:否 - - 字段类型:int - - 默认值:5000 -
- -- **column** - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: -```json -"column": [{ - "name": "col", - "type": "string", - "index":1, - "isPart":false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] -``` - - 属性说明: - - name:必选,字段名称 - - type:必选,字段类型,需要和数据文件中实际的字段类型匹配 - - index:非必选,字段在所有字段中的位置索引,从0开始计算,默认为-1,按照数组顺序依次读取,配置后读取指定字段列 - - isPart:非必选,是否是分区字段,如果是分区字段,会自动从path上截取分区赋值,默认为fale - - format:非必选,按照指定格式,格式化日期 - - value:非必选,常量字段,将value的值作为常量列返回 - - 必选:是 - - 参数类型:数组 - - 默认值:无 - - -### 2、sql - -- **connector** - - 描述:ftp-x - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **path** - - 描述:文件路径 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **protocol** - - 描述:服务器访问协议,目前支持ftp、sftp - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **host** - - 描述:服务地地址 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **port** - - 描述:ftp服务器端口 - - 必选:否 - - 字段类型:int - - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 -
- -- **username** - - 描述:服务器登陆用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **password** - - 描述:服务器登陆密码 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **format** - - 描述:文件的类型,和原生flink保持一致,支持原生所有类型 - - 必选:否 - - 参数类型:string - - 默认值:csv -
- -- **connect-pattern** - - 描述:protocol为ftp时的连接模式,可选PASV和PORT - - 必选:否 - - 字段类型:String - - 默认值:PASV -
- -- **timeout** - - 描述:连接超时时间,单位毫秒 - - 必选:否 - - 字段类型:String - - 默认值:5000 - - -## 五、数据类型 -| 支持 | BOOLEAN | -| --- | --- | -| | TINYINT | -| | SMALLINT | -| | INT | -| | BIGINT | -| | FLOAT | -| | DOUBLE | -| | DECIMAL | -| | STRING | -| | VARCHAR | -| | CHAR | -| | TIMESTAMP | -| | DATE | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹 diff --git a/docs/connectors/ftp/ftp-source.md b/docs/connectors/ftp/ftp-source.md deleted file mode 100644 index 51b389dfce..0000000000 --- a/docs/connectors/ftp/ftp-source.md +++ /dev/null @@ -1,249 +0,0 @@ -# Ftp Source - -## 一、介绍 -ftp source - -## 二、数据源配置 -FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
- -## 三、插件名称 -| sync | ftpsource, ftpreader | -| --- | --- | -| sql | ftp-x | - -## 四、参数说明 - -### 1、sync - -- **path** - - 描述:数据文件路径 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **protocol** - - 描述:服务器访问协议,目前支持ftp、sftp - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **host** - - 描述:ftp服务器地址 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **port** - - 描述:ftp服务器端口 - - 必选:否 - - 字段类型:int - - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 -
- -- **username** - - 描述:ftp服务器登陆用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **password** - - 描述:ftp服务器登陆密码 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **privateKeyPath** - - 描述:sftp私钥文件路径 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **connectPattern** - - 描述:protocol为ftp时的连接模式,可选PASV和PORT,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) - - 必选:否 - - 字段类型:String - - 默认值:PASV -
- -- **fieldDelimiter** - - 描述:读取的字段分隔符 - - 必选:否 - - 字段类型:String - - 默认值:, -
- -- **encoding** - - 描述:读取文件的编码配置 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 -
- -- **controlEncoding** - - 描述:FTP客户端编码格式 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 -
- -- **isFirstLineHeader** - - 描述:首行是否为标题行,如果是则不读取第一行 - - 必选:否 - - 字段类型:boolean - - 默认值:false -
- -- **timeout** - - 描述:连接超时时间,单位毫秒 - - 必选:否 - - 字段类型:String - - 默认值:5000 -
- -- **column** - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: -```json -"column": [{ - "name": "col", - "type": "string", - "index":1, - "isPart":false, - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] -``` - - - 属性说明: - - name:必选,字段名称 - - type:必选,字段类型,需要和数据文件中实际的字段类型匹配 - - index:非必选,字段在所有字段中的位置索引,从0开始计算,默认为-1,按照数组顺序依次读取,配置后读取指定字段列 - - isPart:非必选,是否是分区字段,如果是分区字段,会自动从path上截取分区赋值,默认为fale - - format:非必选,按照指定格式,格式化日期 - - value:非必选,常量字段,将value的值作为常量列返回 - - 必选:是 - - 参数类型:数组 - - 默认值:无 -
- -- **fileType** - - 描述:读取的文件类型,默认取文件后缀名,支持CSV,TXT,EXCEL - - 必选:否 - - 字段类型:string - - 默认值:无 -
- -- **compressType** - - 描述:文件压缩类型,支持ZIP - - 必选:否 - - 字段类型:string - - 默认值:无 -
- -- **fileConfig** - - 描述:文件参数配置 - - 必选:否 - - 字段类型:Map - - 默认值:无 - - 示例: - - csv文件是否进行trim:`"fileConfig":{"trimWhitespace":true}` -
- -#### 2、sql - -- **connector** - - 描述:ftp-x - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **path** - - 描述:文件路径 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **protocol** - - 描述:服务器访问协议,目前支持ftp、sftp - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **host** - - 描述:服务地地址 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **port** - - 描述:ftp服务器端口 - - 必选:否 - - 字段类型:int - - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 -
- -- **username** - - 描述:服务器登陆用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **password** - - 描述:服务器登陆密码 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **format** - - 描述:文件的类型,和原生flink保持一致,支持原生所有类型 - - 必选:否 - - 参数类型:string - - 默认值:csv -
- -- **connect-pattern** - - 描述:protocol为ftp时的连接模式,可选PASV和PORT - - 必选:否 - - 字段类型:String - - 默认值:PASV -
- -- **timeout** - - 描述:连接超时时间,单位毫秒 - - 必选:否 - - 字段类型:String - - 默认值:5000 - - -## 五、数据类型 -| 支持 | BOOLEAN | -| --- | --- | -| | TINYINT | -| | SMALLINT | -| | INT | -| | BIGINT | -| | FLOAT | -| | DOUBLE | -| | DECIMAL | -| | STRING | -| | VARCHAR | -| | CHAR | -| | TIMESTAMP | -| | DATE | - - -## 六、配置示例 -见项目内`chunjun-examples`文件夹 diff --git a/docs/connectors/hbase/hbase-lookup.md b/docs/connectors/hbase/hbase-lookup.md deleted file mode 100644 index c69c90142f..0000000000 --- a/docs/connectors/hbase/hbase-lookup.md +++ /dev/null @@ -1,138 +0,0 @@ -# HBase Lookup - -## 一、介绍 -HBase维表,支持全量和异步方式
-全量缓存:将维表数据全部加载到内存中,建议数据量不大,且数据不经常变动的场景使用。
-异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 - -## 二、支持版本 -HBase 1.4 + - - -## 三、插件名称 -| SQL | hbase14-x | -| --- | --- | - -## 四、参数说明 -- **connector** - - 描述:hbase14-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
-- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
-- **zookeeper.quorum** - - 描述:HBase的Zookeeper地址 - - 必选:是 - - 参数类型:String - - 默认值:无 -
-- **zookeeper.znode.parent** - - 描述:root dir - - 必选:是 - - 参数类型:String - - 默认值:/hbase -
- -- **null-string-literal** - - 描述:当字符串值为 null 时的存储形式 - - 必选:是 - - 参数类型:String - - 默认值:null -
-- **properties.*** - - 描述:HBase原生选项 如'properties.hbase.security.authentication' = 'kerberos'. - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **lookup.max-retries** - - 描述:the max retry times if lookup database failed. - - 必选:否 - - 参数类型:int - - 默认值:3 -
- -- **lookup.async-timeout** - - 描述:async timeout. - - 必选:否 - - 参数类型:int - - 默认值:10000 -
- -- **lookup.error-limit** - - 描述:errorLimit - - 必选:是 - - 参数类型:Long.MAX_VALUE - - 默认值:无 -
- -- **lookup.cache-type** - - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - - 必选:否 - - 参数类型:string - - 默认值:LRU -
- -- **lookup.cache-period** - - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - - 必选:否 - - 参数类型:string - - 默认值:3600000 -
- -- **lookup.cache.max-rows** - - 描述:lru维表缓存数据的条数,默认10000条 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **lookup.cache.ttl** - - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - - 必选:否 - - 参数类型:string - - 默认值:60000 -
- -- **lookup.fetch-size** - - 描述:ALL维表每次从数据库加载的条数,默认1000条 - - 必选:否 - - 参数类型:string - - 默认值:1000 -
- -- **lookup.parallelism** - - 描述:维表并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **security.kerberos.principal** - - 描述:kerberos认证的principal - - 必选:是 - - 默认值:无 -- **security.kerberos.keytab** - - 描述:kerberos认证的keytab文件路径 - - 必选:是 - - 默认值:无 -- **security.kerberos.krb5conf** - - 描述:kerberos认证的krb5conf文件路径 - - 必选:是 - - 默认值:无 - -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/hbase/hbase-sink.md b/docs/connectors/hbase/hbase-sink.md deleted file mode 100644 index d3f46e0be2..0000000000 --- a/docs/connectors/hbase/hbase-sink.md +++ /dev/null @@ -1,215 +0,0 @@ -# HBase Sink - -## 一、介绍 -HBase sink - -## 二、支持版本 -HBase 1.4 + - - -## 三、插件名称 -| Sync | hbasesink、hbasewriter | -| --- | --- | -| SQL | hbase14-x | - - -## 四、参数说明 -### 1、Sync -- **tablename** - - 描述:hbase表名 - - 必选:是 - - 默认值:无 - - - -- **hbaseConfig** - - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: - -Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal - -- 必选:是 -- 默认值:无 - - - -- **nullMode** - - 描述:读取的null值时,如何处理。支持两种方式: - - skip:表示不向hbase写这列; - - empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0] - - 必选:否 - - 默认值:skip - - - -- **encoding** - - 描述:字符编码 - - 必选:无 - - 默认值:UTF-8 - -
- -- **walFlag** - - 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能 - - 必选:否 - - 默认值:false - - - -- **writeBufferSize** - - 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求 - - 必选:否 - - 默认值:8388608(8M) - - - -- **scanCacheSize** - - 描述:一次RPC请求批量读取的Results数量 - - 必选:无 - - 默认值:256 - - - -- **scanBatchSize** - - 描述:每一个result中的列的数量 - - 必选:无 - - 默认值:100 - - - -- **column** - - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; - - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 - - 必选:是 - - 默认值:无 - - - -- **rowkeyColumn** - - 描述:用于构造rowkey的描述信息,支持两种格式,每列形式如下 - - 字符串格式 -
字符串格式为:$(cf:col),可以多个字段组合:$(cf:col1)_$(cf:col2), -
可以使用md5函数:md5($(cf:col)) - - 数组格式 - - 普通列 -``` -{ - "index": 0, // 该列在column属性中的序号,从0开始 - "type": "string" 列的类型,默认为string -} -``` - - - 常数列 -``` -{ - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string -} -``` - -- 必选:否 -
如果不指定idColumns属性,则会随机产生文档id -- 默认值:无 - - - -- **versionColumn** - - 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: -``` -"versionColumn":{ -"index":1 -} -``` - --
或者 -``` -"versionColumn":{ -"value":"123456789" -} -``` - -
- - - -### 2、SQL -- **connector** - - 描述:hbase14-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
-- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
-- **zookeeper.quorum** - - 描述:HBase的Zookeeper地址 - - 必选:是 - - 参数类型:String - - 默认值:无 -
-- **zookeeper.znode.parent** - - 描述:root dir - - 必选:是 - - 参数类型:String - - 默认值:/hbase -
- -- **null-string-literal** - - 描述:当字符串值为 null 时的存储形式 - - 必选:是 - - 参数类型:String - - 默认值:null -
- -- **properties.*** - - 描述:HBase原生选项 如'properties.hbase.security.authentication' = 'kerberos'. - - 必选:是 - - 参数类型:String - - 默认值:无 -
-- **sink.buffer-flush.max-rows** - - 描述:批量写数据条数,单位:条 - - 必选:否 - - 参数类型:String - - 默认值:1024 -
- -- **sink.buffer-flush.interval** - - 描述:批量写时间间隔,单位:时间 - - 必选:否 - - 参数类型:Duration - - 默认值:10000 -
- -- **sink.parallelism** - - 描述:写入结果的并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **security.kerberos.principal** - - 描述:kerberos认证的principal - - 必选:是 - - 默认值:无 -- **security.kerberos.keytab** - - 描述:kerberos认证的keytab文件路径 - - 必选:是 - - 默认值:无 -- **security.kerberos.krb5conf** - - 描述:kerberos认证的krb5conf文件路径 - - 必选:是 - - 默认值:无 - -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/hbase/hbase-source.md b/docs/connectors/hbase/hbase-source.md deleted file mode 100644 index b6256f7493..0000000000 --- a/docs/connectors/hbase/hbase-source.md +++ /dev/null @@ -1,161 +0,0 @@ -# HBase Source - -## 一、介绍 -支持从HBase离线读取,支持HBase实时间隔轮询读取 - -## 二、支持版本 -HBase 1.4 + - - -## 三、插件名称 -| Sync | hbasesource、hbasereader | -| --- | --- | -| SQL | hbase1.4-x | - - -## 四、参数说明 -### 1、Sync -- **table** - - 描述:hbase表名 - - 必选:是 - - 默认值:无 - ``` -
- -- **hbaseConfig** - - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: - -Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal - -- 必选:是 -- 默认值:无 - -- **range** - - 描述:指定hbasereader读取的rowkey范围。 - - startRowkey:指定开始rowkey; - - endRowkey:指定结束rowkey; - - - - isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey),配置格式如下: -``` -"range": { - "startRowkey": "aaa", - "endRowkey": "ccc", - "isBinaryRowkey":false -} -``` - -- 注意:如果用户配置了 startRowkey 和 endRowkey,需要确保:startRowkey <= endRowkey -- 必选:否 -- 默认值:无 - - - -- **encoding** - - 描述:字符编码 - - 必选:无 - - 默认值:无 - - -- **scanCacheSize** - - 描述:一次RPC请求批量读取的Results数量 - - 必选:无 - - 默认值:256 - -
- -- **scanBatchSize** - - 描述:每一个result中的列的数量 - - 必选:无 - - 默认值:100 - -
- -- **column** - - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; - - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 - - 必选:是 - - 默认值:无 - - -
- - -## 四、配置示例 -```json -{ - "job": { - "content": [ - { - "reader": { - "name": "hbasereader", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "node01,node02,node03", - "zookeeper.znode.parent": "/hbase" - }, - "table": "sb5", - "encodig": "utf-8", - "column": [ - { - "name": "rowkey", - "type": "string" - }, - { - "name": "cf1:id", - "type": "string" - } - ], - "range": { - "startRowkey": "", - "endRowkey": "", - "isBinaryRowkey": true - } - } - }, - "writer": { - "parameter": { - "print": true - }, - "name": "streamwriter" - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log": { - "isLogger": false, - "level": "debug", - "path": "", - "pattern": "" - } - } - } -} -``` - -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/http/http-sink.md b/docs/connectors/http/http-sink.md deleted file mode 100644 index dae58bca09..0000000000 --- a/docs/connectors/http/http-sink.md +++ /dev/null @@ -1,87 +0,0 @@ -# Http sink - -## 一、介绍 -Http sink - -## 二、插件名称 - -| Mode | Name | -| --- | --- | -| SYNC | httpsink, httpwriter | -| SQL | http-x | - - - -## 三、参数说明 - -### 1、Sync -- **url** - - 描述:请求url地址 - - 必选:是 - - 字段类型:String - - 默认值:无 - -- **method** - - 描述:请求方式 - - 必选:否 - - 字段类型:String - - 默认值:post - -- **header** - - 描述:请求头参数 - - 必选:否 - - 字段类型:Map - - 默认值:无 - -- **column** - - 描述:用户请求自定义参数 - - 必选:否 - - 字段类型:List - - 默认值:无 - -- **body** - - 描述:对应post请求的body参数 - - 必选:否 - - 字段类型:数组 - - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算), - - 内置变量 - - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 - - ${intervalTime}间隔时间,代表参数 intervalTime 的值 - - ${uuid} 随机字符串 32位的随机字符串 - - param/body/response变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 参数解析 - - name:请求的key,必选 - - value:请求key对应的值,必选 - - nextValue:除第一次请求之外,请求Key对应的值,非必选; - - format:格式化模版,非必选,如果请求体是时间格式,则为必选; - - 默认值:无 - -- **param** - - 描述:对应get请求参数 - - 必选:否 - - 字段类型:数组 - - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算) - - 内置变量 - - ${currentTime}:获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型; - - ${intervalTime}:间隔时间,代表参数 intervalTime 的值; - - ${uuid} 随机字符串 32位的随机字符串 - - param/body/response变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 参数解析 - - name:请求的key,必选 - - value:请求key对应的值,必选 - - nextValue:除第一次请求之外,请求Key对应的值,非必选; - - format:格式化模版,非必选,如果请求体是时间格式,则为必选; - - 默认值:无 - - -### 2、SQL - - -## 四、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/http/http-source.md b/docs/connectors/http/http-source.md deleted file mode 100644 index d34ab0720a..0000000000 --- a/docs/connectors/http/http-source.md +++ /dev/null @@ -1,175 +0,0 @@ -# Http Source - -## 一、介绍 - -http source - -## 二、插件名称 - -| Mode | Name | -| --- | --- | -| SYNC | httpsource, httpreader | -| SQL | http-x | - -## 三、参数说明 - -### 1、Sync - -- **protocol** - - 描述:http请求协议 - - 必选:否 - - 字段类型:String - - 默认值:https - -- **decode** - - 描述:解码器返回数据,是作为json格式,还是text格式处理; - - text:不做任何处理,直接返回; - - json:可以进行定制化输出,指定输出的key,则对返回值解析,获取对应的key以及值 组装新的json数据丢出去; - - 必选:否 - - 字段类型:String - - 默认值:text - -- **fields** - - 描述:在decode为json时,可以对返回值指定key输出;当decode为text时,不支持此参数;key以'.'为层级,多个key以','隔开; - - 必选:否 - - 字段类型:String - - 默认值:无 - - 示例: - - fields值为:"fields": "msg.key1,msg.key2.key3" - - 返回值为:{"msg":{"key1":"value1","key2":{"key3":"value2","key4":"value3"},"key5":2}} - - 根据fields解析之后的值为:{"msg":{"key1":"value1","key2":{"key3":"value2"}}} - -- **strategy** - - 描述:定义的key的实际值与value指定值相等时进行对应的逻辑处理;针对返回类型为json的数据,用户会指定key以及对应的value和处理方式。如果返回数据的对应的key的值正好和用户配置的value相等,则执行对应逻辑。同时用户指定的key可以来自返回值也可以来自param参数值 - - 必选:否 - - 字段类型:数组 - - 默认值:无 - - 示例:"strategy":[{"key":"${param.pageNumber}","value":"${response.totalPageNum}","handle":"stop"}] - - 参数解析 - - key 选择对应参数的key,支持的格式为 - - 变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 内置变量 - - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 - - ${intervalTime}间隔时间,代表参数 intervalTime 的值 - - ${uuid} 随机字符串 32位的随机字符串 - - value 匹配的值,支持的格式为: - - 常量 - - 变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 内置变量 - - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 - - ${intervalTime}间隔时间,代表参数 intervalTime 的值 - - ${uuid} 随机字符串 32位的随机字符串 - - handle 对应处理逻辑 - - stop:停止任务; - - retry:重试,如果三次重试都失败,则任务失败; - -- **intervalTime** - - 描述:用户请求间隔时间,单位毫秒 - - 必选:是 - - 字段类型:Long - - 默认值:无 - -- **url** - - 描述:请求url地址 - - 必选:是 - - 字段类型:String - - 默认值:无 - -- **method** - - 描述:请求方式 - - 必选:否 - - 字段类型:String - - 默认值:post - -- **header** - - 描述:请求头参数 - - 必选:否 - - 字段类型:Map - - 默认值:无 - -- **column** - - 描述:用户请求自定义参数 - - 必选:否 - - 字段类型:List - - 默认值:无 - -- **body** - - 描述:对应post请求的body参数 - - 必选:否 - - 字段类型:数组 - - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算), - - 内置变量 - - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 - - ${intervalTime}间隔时间,代表参数 intervalTime 的值 - - ${uuid} 随机字符串 32位的随机字符串 - - param/body/response变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 参数解析 - - name:请求的key,必选 - - value:请求key对应的值,必选 - - nextValue:除第一次请求之外,请求Key对应的值,非必选; - - format:格式化模版,非必选,如果请求体是时间格式,则为必选; - - 默认值:无 - -- **param** - - 描述:对应get请求参数 - - 必选:否 - - 字段类型:数组 - - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算) - - 内置变量 - - ${currentTime}:获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型; - - ${intervalTime}:间隔时间,代表参数 intervalTime 的值; - - ${uuid} 随机字符串 32位的随机字符串 - - param/body/response变量 - - ${param.key} 对应get请求param参数里key对应的值 - - ${body.key}对应post请求的body参数里key对应的值 - - ${response.key} 对应返回值里的key对应的值 - - 参数解析 - - name:请求的key,必选 - - value:请求key对应的值,必选 - - nextValue:除第一次请求之外,请求Key对应的值,非必选; - - format:格式化模版,非必选,如果请求体是时间格式,则为必选; - - 默认值:无 - -### 2、SQL - -- **url** - - 描述:请求url地址 - - 必选:是 - - 字段类型:String - - 默认值:无 - -- **method** - - 描述:请求方式 - - 必选:否 - - 字段类型:String - - 默认值:post - -- **header** - - 描述:请求头参数 - - 必选:否 - - 字段类型:Map - - 默认值:无 - -- **column** - - 描述:用户请求自定义参数 - - 必选:否 - - 字段类型:List - - 默认值:无 - -## 四、数据类型 - -和原生flink数据类型保持一致
每种format所支持的数据类型请参考[flink官方文档](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/formats/)
- -## 五、脚本示例 - -见项目内`chunjun-examples`文件夹。 - diff --git a/docs/connectors/kafka/kafka-sink.md b/docs/connectors/kafka/kafka-sink.md deleted file mode 100644 index 3f59e0bf3c..0000000000 --- a/docs/connectors/kafka/kafka-sink.md +++ /dev/null @@ -1,198 +0,0 @@ -# kafka sink - -## 一、介绍 -kafka sink - -## 二、支持版本 -kafka主流版本 - - -## 三、插件名称 -| Sync | kafkasink、kafkawriter | -| --- | --- | -| SQL | kafka-x | - - -## 四、参数说明 -### 1、Sync -- **topic** - - 描述:消息发送至kafka的topic名称,不支持多个topic - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **producerSettings** - - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 - - 必选:是 - - 字段类型:Map - - 默认值:无 - - 如: - ```json - { - "producerSettings":{ - "bootstrap.servers":"host1:9092,host2:9092,host3:9092" - } - } - ``` -
- -- **tableFields** - - 描述:字段映射配置。从reader插件传递到writer插件的的数据只包含其value属性,配置该参数后可将其还原成键值对类型json字符串输出。 - - 注意: - - 若配置该属性,则该配置中的字段个数必须不少于reader插件中读取的字段个数,否则该配置失效; - - 映射关系按该配置中字段的先后顺序依次匹配; - - 必选:否 - - 字段类型:String[] - - 默认值:无 -
- -### 2、SQL -具体可以参考:[kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) -- **connector** - - 描述:kafka-x - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **topic** - - 描述:当表用作源时要从中读取数据的主题名称。它还通过用分号分隔主题来支持源的主题列表,如'topic-1;topic-2'. 请注意,只能为源指定“topic-pattern”和“topic”之一。当表用作接收器时,主题名称是要写入数据的主题。接收器不支持注意主题列表。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **topic-pattern** - - 描述:要从中读取的主题名称模式的正则表达式。当作业开始运行时,消费者将订阅名称与指定正则表达式匹配的所有主题。请注意,只能为源指定“topic-pattern”和“topic”之一。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **properties.bootstrap.servers** - - 描述:逗号分隔的 Kafka 代理列表。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **properties.group.id** - - 描述:Kafka source的消费组id,Kafka sink可选。 - - 必选:required by source - - 字段类型:String - - 默认值:无 -
- -- **properties.*** - - 描述:这可以设置和传递任意 Kafka 配置。后缀名称必须与[Kafka 配置文档中](https://kafka.apache.org/documentation/#configuration)定义的配置键匹配。Flink 将删除“属性”。键前缀并将转换后的键和值传递给底层 KafkaClient。例如,您可以通过 禁用自动主题创建'properties.allow.auto.create.topics' = 'false'。但是有一些配置是不支持设置的,因为 Flink 会覆盖它们。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **format** - - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:此选项或'value.format'选项都是必需的。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **key.format** - - 描述:用用于反序列化和序列化 Kafka 消息关键部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:如果定义了密钥格式,则该'key.fields' 选项也是必需的。否则 Kafka 记录将有一个空键。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **key.fields** - - 描述:定义表架构中物理列的显式列表,用于配置键格式的数据类型。默认情况下,此列表为空,因此未定义键。该列表应如下所示'field1;field2'。 - - 必选:否 - - 字段类型:List - - 默认值:无 -
- -- **key.fields-prefix** - - 描述:为键格式的所有字段定义自定义前缀,以避免与值格式的字段发生名称冲突。默认情况下,前缀为空。如果定义了自定义前缀,则表架构 和'key.fields'都将使用前缀名称。在构造密钥格式的数据类型时,将删除前缀,并在密钥格式中使用非前缀名称。请注意,此选项要求'value.fields-include' 必须设置为'EXCEPT_KEY'。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **value.format** - - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多格式选项,请参阅格式页面。注意:此选项或'format'[选项](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)都是必需的。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **value.fields-include** - - 描述:定义如何处理值格式的数据类型中的键列的策略。默认情况下,'ALL'表模式的物理列将包含在值格式中,这意味着键列出现在键和值格式的数据类型中 - - 必选:否 - - 字段类型:枚举 - - 可选的值:[ALL, EXCEPT_KEY] - - 默认值:ALL -
- -- **scan.startup.mode** - - 描述:卡夫卡消费的启动模式,有效值为'earliest-offset','latest-offset','group-offsets','timestamp'和'specific-offsets'。有关更多详细信息,请参阅以下[开始阅读位置](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#start-reading-position)。 - - 必选:否 - - 字段类型:String - - 默认值:group-offsets -
- -- **scan.startup.specific-offsets** - - 描述:在'specific-offsets'启动模式下为每个分区指定偏移量,例如'partition:0,offset:42;partition:1,offset:300'. - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **scan.startup.timestamp-millis** - - 描述:从'timestamp'启动模式下使用的指定纪元时间戳(毫秒)开始。 - - 必选:否 - - 字段类型:Long - - 默认值:无 -
- -- **scan.topic-partition-discovery.interval** - - 描述:消费者定期发现动态创建的 Kafka 主题和分区的时间间隔。 - - 必选:否 - - 字段类型:Duration - - 默认值:无 -
- -- **sink.partitioner** - - 描述: 从 Flink 的分区到 Kafka 的分区的输出分区。有效值为 - - default: 使用 kafka 默认分区器对记录进行分区。 - - fixed:每个 Flink 分区最终最多包含一个 Kafka 分区。 - - round-robin:一个 Flink 分区被分发到 Kafka 分区粘性循环。它仅在未指定记录的键时有效。 - - 自定义FlinkKafkaPartitioner子类:例如'org.mycompany.MyPartitioner'. - - 有关更多详细信息,请参阅以下[接收器分区](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#sink-partitioning)。 - - 必选:否 - - 字段类型:String - - 默认值:default -
- -- **sink.semantic** - - 描述:定义 Kafka 接收器的交付语义。有效的枚举是'at-least-once','exactly-once'和'none'。有关更多详细信息,请参阅[一致性保证](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#consistency-guarantees)。 - - 必选:否 - - 字段类型:String - - 默认值:at-least-once -
- -- **sink.parallelism** - - 描述:定义 Kafka sink 操作符的并行性。默认情况下,并行度由框架使用与上游链式运算符相同的并行度确定。 - - 必选:否 - - 字段类型:Integer - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| 暂不支持 | 其他 | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/kafka/kafka-source.md b/docs/connectors/kafka/kafka-source.md deleted file mode 100644 index 86f95b9473..0000000000 --- a/docs/connectors/kafka/kafka-source.md +++ /dev/null @@ -1,287 +0,0 @@ -# Kafka Source - -## 一、介绍 -Kafka Source - -## 二、支持版本 -kafka主流版本 - - -## 三、插件名称 -| Sync | kafkasource、kafkareader | -| --- | --- | -| SQL | kafka-x | - - -## 四、参数说明 -### 1、Sync -- **topic** - - 描述:要消费的topic,多个以,分割,当`mode`为`timestamp`、`specific-offsets`时不支持多topic - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **mode** - - 描述:kafka消费端启动模式,目前仅支持`kafkareader`插件 - - 可选值: - - group-offsets:     从ZK / Kafka brokers中指定的消费组已经提交的offset开始消费 - - earliest-offset:    从最早的偏移量开始(如果可能) - - latest-offset:     从最新的偏移量开始(如果可能) - - timestamp:         从每个分区的指定的时间戳开始 - - specific-offsets: 从每个分区的指定的特定偏移量开始 - - 必选:否 - - 字段类型:String - - 默认值:group-offsets -
- -- **timestamp** - - 描述:指定的kafka时间戳采集起点,目前仅支持`kafkareader`插件 - - 必选:当`mode`为`timestamp`时必选 - - 字段类型:Long - - 默认值:无 -
- -- **offset** - - 描述:消费的分区及对应的特定偏移量,目前仅支持`kafkareader`插件 - - 必选:当`mode`为`specific-offsets`时必选 - - 字段类型:String - - 格式:partition:0,offset:42;partition:1,offset:300;partition:2,offset:300 - - 默认值:无 -
- -- **groupId** - - 描述:kafka消费组Id - - 必选:否 - - 字段类型:String - - 默认值:default -
- -- **encoding** - - 描述:字符编码 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 -
- -- **codec** - - 描述:编码解码器类型,支持 json、text - - text: - 将kafka获取到的消息字符串存储到一个key为message的map中,如:kafka中的消息为:{"key":"key","message":"value"}, - 则发送至下游的数据格式为: - ```json - [ - { - "message":"{\"key\": \"key\", \"value\": \"value\"}" - } - ] - ``` - - json:将kafka获取到的消息字符串按照json格式进行解析 - - 若该字符串为json格式 - - 当其中含有message字段时,发送至下游的数据格式为: - ```json - [ - { - "key":"key", - "message":"value" - } - ] - ``` - - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,发送至下游的数据格式为: - ```json - [ - { - "key":"key", - "value":"value", - "message":"{\"key\": \"key\", \"value\": \"value\"}" - } - ] - ``` - - 若改字符串不为json格式,则按照text类型进行处理 - - 必选:否 - - 字段类型:String - - 默认值:text -
- -- **consumerSettings** - - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 - - 必选:是 - - 字段类型:Map - - 默认值:无 - - 如: - ```json - { - "consumerSettings":{ - "bootstrap.servers":"host1:9092,host2:9092,host3:9092" - } - } - ``` - -- **column** - - 描述:kafka向MySQL写数据时,对应MySQL表中的字段名 - - 必选:否 - - 字段类型:List - - 默认值:无 - - 注意:需指定字段的具体信息,属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - 如: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss" - }] - ``` -
- -### 2、SQL -具体可以参考:[kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) -- **connector** - - 描述:kafka-x - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **topic** - - 描述:当表用作源时要从中读取数据的主题名称。它还通过用分号分隔主题来支持源的主题列表,如'topic-1;topic-2'. 请注意,只能为源指定“topic-pattern”和“topic”之一。当表用作接收器时,主题名称是要写入数据的主题。接收器不支持注意主题列表。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **topic-pattern** - - 描述:要从中读取的主题名称模式的正则表达式。当作业开始运行时,消费者将订阅名称与指定正则表达式匹配的所有主题。请注意,只能为源指定“topic-pattern”和“topic”之一。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **properties.bootstrap.servers** - - 描述:逗号分隔的 Kafka 代理列表。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **properties.group.id** - - 描述:Kafka source的消费组id,Kafka sink可选。 - - 必选:required by source - - 字段类型:String - - 默认值:无 -
- -- **properties.*** - - 描述:这可以设置和传递任意 Kafka 配置。后缀名称必须与[Kafka 配置文档中](https://kafka.apache.org/documentation/#configuration)定义的配置键匹配。Flink 将删除“属性”。键前缀并将转换后的键和值传递给底层 KafkaClient。例如,您可以通过 禁用自动主题创建'properties.allow.auto.create.topics' = 'false'。但是有一些配置是不支持设置的,因为 Flink 会覆盖它们。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **format** - - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:此选项或'value.format'选项都是必需的。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **key.format** - - 描述:用用于反序列化和序列化 Kafka 消息关键部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:如果定义了密钥格式,则该'key.fields' 选项也是必需的。否则 Kafka 记录将有一个空键。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **key.fields** - - 描述:定义表架构中物理列的显式列表,用于配置键格式的数据类型。默认情况下,此列表为空,因此未定义键。该列表应如下所示'field1;field2'。 - - 必选:否 - - 字段类型:List - - 默认值:无 -
- -- **key.fields-prefix** - - 描述:为键格式的所有字段定义自定义前缀,以避免与值格式的字段发生名称冲突。默认情况下,前缀为空。如果定义了自定义前缀,则表架构 和'key.fields'都将使用前缀名称。在构造密钥格式的数据类型时,将删除前缀,并在密钥格式中使用非前缀名称。请注意,此选项要求'value.fields-include' 必须设置为'EXCEPT_KEY'。 - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **value.format** - - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多格式选项,请参阅格式页面。注意:此选项或'format'[选项](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)都是必需的。 - - 必选:是 - - 字段类型:String - - 默认值:无 -
- -- **value.fields-include** - - 描述:定义如何处理值格式的数据类型中的键列的策略。默认情况下,'ALL'表模式的物理列将包含在值格式中,这意味着键列出现在键和值格式的数据类型中 - - 必选:否 - - 字段类型:枚举 - - 可选的值:[ALL, EXCEPT_KEY] - - 默认值:ALL -
- -- **scan.startup.mode** - - 描述:卡夫卡消费的启动模式,有效值为'earliest-offset','latest-offset','group-offsets','timestamp'和'specific-offsets'。有关更多详细信息,请参阅以下[开始阅读位置](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#start-reading-position)。 - - 必选:否 - - 字段类型:String - - 默认值:group-offsets -
- -- **scan.startup.specific-offsets** - - 描述:在'specific-offsets'启动模式下为每个分区指定偏移量,例如'partition:0,offset:42;partition:1,offset:300'. - - 必选:否 - - 字段类型:String - - 默认值:无 -
- -- **scan.startup.timestamp-millis** - - 描述:从'timestamp'启动模式下使用的指定纪元时间戳(毫秒)开始。 - - 必选:否 - - 字段类型:Long - - 默认值:无 -
- -- **scan.topic-partition-discovery.interval** - - 描述:消费者定期发现动态创建的 Kafka 主题和分区的时间间隔。 - - 必选:否 - - 字段类型:Duration - - 默认值:无 -
- -- **sink.partitioner** - - 描述: 从 Flink 的分区到 Kafka 的分区的输出分区。有效值为 - - default: 使用 kafka 默认分区器对记录进行分区。 - - fixed:每个 Flink 分区最终最多包含一个 Kafka 分区。 - - round-robin:一个 Flink 分区被分发到 Kafka 分区粘性循环。它仅在未指定记录的键时有效。 - - 自定义FlinkKafkaPartitioner子类:例如'org.mycompany.MyPartitioner'. - - 有关更多详细信息,请参阅以下[接收器分区](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#sink-partitioning)。 - - 必选:否 - - 字段类型:String - - 默认值:default -
- -- **sink.semantic** - - 描述:定义 Kafka 接收器的交付语义。有效的枚举是'at-least-once','exactly-once'和'none'。有关更多详细信息,请参阅[一致性保证](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#consistency-guarantees)。 - - 必选:否 - - 字段类型:String - - 默认值:at-least-once -
- -- **sink.parallelism** - - 描述:定义 Kafka sink 操作符的并行性。默认情况下,并行度由框架使用与上游链式运算符相同的并行度确定。 - - 必选:否 - - 字段类型:Integer - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | -| --- | --- | -| 暂不支持 | 其他 | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/logminer/LogMiner-source.md b/docs/connectors/logminer/LogMiner-source.md deleted file mode 100644 index 4d1d9f3680..0000000000 --- a/docs/connectors/logminer/LogMiner-source.md +++ /dev/null @@ -1,387 +0,0 @@ -# LogMiner Source - - - -- [LogMiner Source](#logminer-source) - - [一、介绍](#一介绍) - - [二、支持版本](#二支持版本) - - [三、数据库配置](#三数据库配置) - - [四、LogMiner原理](#四logminer原理) - - [五、插件名称](#五插件名称) - - [六、参数说明](#六参数说明) - - [1、Sync](#1sync) - - [2、SQL](#2sql) - - [七、数据结构](#七数据结构) - - [八、数据类型](#八数据类型) - - [九、脚本示例](#九脚本示例) - - - -
- -## 一、介绍 -OracleLogMiner插件支持配置监听表名称以及读取起点读取日志数据。OracleLogMiner在checkpoint时保存当前消费的位点,因此支持续跑。 -
- -## 二、支持版本 -oracle10,oracle11,oracle12,oracle19,支持RAC,主备架构 -
- -## 三、数据库配置 -[Oracle配置LogMiner](LogMiner配置.md) -
- -## 四、LogMiner原理 -[LogMiner原理](LogMiner原理.md) -
- -## 五、插件名称 -| Sync | oraclelogminerreader、oraclelogminersource | -| --- | --- | -| SQL | oraclelogminer-x | - -
- -## -## 六、参数说明 -### 1、Sync - -- **jdbcUrl** - - 描述:oracle数据库的JDBC URL链接 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- - -- **username** - - 描述:用户名 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **password** - - 描述:密码 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **table** - - 描述: 需要监听的表,格式为:schema.table,schema不能配置为*,但table可以配置*监听指定库下所有的表,如:["schema1.table1","schema1.table2","schema2.*"] - - 必选:否,不配置则监听除`SYS`库以外的所有库的所有表变更信息 - - 字段类型:数组 - - 默认值:无 - -
- - -- **splitUpdate** - - 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见【七、数据结构说明】 - - 必选:否 - - 字段类型:boolean - - 默认值:false - -
- -- **cat** - - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 - - 必选:否 - - 字段类型:String - - 默认值:UPDATE,INSERT,DELETE - -
- -- **readPosition** - - 描述:Oracle实时采集的采集起点 - - 可选值: - - all: 从Oracle数据库中最早的归档日志组开始采集(不建议使用) - - current:从任务运行时开始采集 - - time: 从指定时间点开始采集 - - scn: 从指定SCN号处开始采集 - - 必选:否 - - 字段类型:String - - 默认值:current - - -
- - -- **startTime** - - 描述: 指定采集起点的毫秒级时间戳 - - 必选:当`readPosition`为`time`时,该参数必填 - - 字段类型:Long(毫秒级时间戳) - - 默认值:无 - -
- -- **startSCN** - - 描述: 指定采集起点的SCN号 - - 必选:当`readPosition`为`scn`时,该参数必填 - - 字段类型:String - - 默认值:无 - -
- -- **fetchSize** - - 描述: 批量从v$logmnr_contents视图中拉取的数据条数,对于大数据量的数据变更,调大该值可一定程度上增加任务的读取速度 - - 必选:否 - - 字段类型:Integer - - 默认值:1000 - -
- -- **queryTimeout** - - 描述: LogMiner执行查询SQL的超时参数,单位秒 - - 必选:否 - - 字段类型:Long - - 默认值:300 - -
- -- **supportAutoAddLog** - - 描述:启动LogMiner是否自动添加日志组(不建议使用) - - 必选:否 - - 字段类型:Boolean - - 默认值:false - -
- -- **pavingData** - - 描述:是否将解析出的json数据拍平,具体见【七、数据结构说明】 - - 必选:否 - - 字段类型:boolean - - 默认值:false - -
- -### 2、SQL -- **url** - - 描述:oracle数据库的JDBC URL链接 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **username** - - 描述:用户名 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **password** - - 描述:密码 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **table** - - 描述:需要解析的数据表。 - - 注意:SQL任务只支持监听单张表,且数据格式为schema.table - - 必选:否 - - 字段类型:string - - 默认值:无 - -
- -- **cat** - - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 - - 必选:否 - - 字段类型:String - - 默认值:UPDATE,INSERT,DELETE - -
- -- **read-position** - - 描述:Oracle实时采集的采集起点 - - 可选值: - - all: 从Oracle数据库中最早的归档日志组开始采集(不建议使用) - - current:从任务运行时开始采集 - - time: 从指定时间点开始采集 - - scn: 从指定SCN号处开始采集 - - 必选:否 - - 字段类型:String - - 默认值:current - -
- -- **start-time** - - 描述: 指定采集起点的毫秒级时间戳 - - 必选:当`readPosition`为`time`时,该参数必填 - - 字段类型:Long(毫秒级时间戳) - - 默认值:无 - -
- -- **start-scn** - - 描述: 指定采集起点的SCN号 - - 必选:当`readPosition`为`scn`时,该参数必填 - - 字段类型:String - - 默认值:无 - -
- -- **fetch-size** - - 描述: 批量从v$logmnr_contents视图中拉取的数据条数,对于大数据量的数据变更,调大该值可一定程度上增加任务的读取速度 - - 必选:否 - - 字段类型:Integer - - 默认值:1000 - -
- -- **query-timeout** - - 描述: LogMiner执行查询SQL的超时参数,单位秒 - - 必选:否 - - 字段类型:Long - - 默认值:300 - -
- -- **support-auto-add-log** - - 描述:启动LogMiner是否自动添加日志组(不建议使用) - - 必选:否 - - 字段类型:Boolean - - 默认值:false - -
- -- **io-threads** - - 描述:IO处理线程数,最大线程数为3 - - 必选:否 - - 字段类型:int - - 默认值:1 - -
- -- **max-log-file-size** - - 描述:logminer一次性加载的日志文件的大小,默认5g,单位byte - - 必选:否 - - 字段类型:long - - 默认值:5*1024*1024*1024 - -
- -- **transaction-cache-num-size** - - 描述:logminer可缓存DML的数量 - - 必选:否 - - 字段类型:long - - 默认值:800 - -
- -- **transaction-expire-time** - - 描述:logminer可缓存的失效时间,单位分钟 - - 必选:否 - - 字段类型:int - - 默认值:20 - -
-## 七、数据结构 -在2021-06-29 23:42:19(时间戳:1624981339000)执行: -```sql -INSERT INTO TIEZHU.RESULT1 ("id", "name", "age") VALUES (1, 'a', 12) -``` -
- -在2021-06-29 23:42:29(时间戳:1624981349000)执行: -```sql -UPDATE TIEZHU.RESULT1 t SET t."id" = 2, t."age" = 112 WHERE t."id" = 1 -``` -
- -在2021-06-29 23:42:34(时间戳:1624981354000)执行: -```sql - DELETE FROM TIEZHU.RESULT1 WHERE "id" = 2 -``` -
- -1、pavingData = true, splitUpdate = false -RowData中的数据依次为: -``` -//scn schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a", 12] -[49982969,"TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", 1, "a", 12 , 2, "a", 112] -[49982973,"TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a",112 , null, null, null] -``` -
- -2、pavingData = false, splitUpdate = false -RowData中的数据依次为: -``` -//scn, schema, table, ts, opTime, type, before, after -[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", {"id":1, "name":"a", "age":12}, {"id":2, "name":"a", "age":112}] -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] -``` -
- -3、pavingData = true, splitUpdate = true -RowData中的数据依次为: -``` -//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a",12 ] - -//scn, schema, table, opTime, ts, type, before_id, before_name, before_age -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", 1, "a", 12] -//scn, schema, table, opTime, ts, type, after_id, after_name, after_age -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", 2, "a", 112] - -//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a", 112, null, null, null] - - -``` -
- -4、pavingData = false, splitUpdate = true -RowData中的数据依次为: -``` -//scn, schema, table, ts, opTime, type, before, after -[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] -//scn, schema, table, ts, opTime, type, before -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", {"id":1, "name":"a", "age":12}] -//scn, schema, table, ts, opTime, type, after -[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", {"id":2, "name":"a", "age":112}] -//scn, schema, table, ts, opTime, type, before, after -[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] - -``` -
- -- scn:Oracle数据库变更记录对应的scn号 -- type:变更类型,INSERT,UPDATE、DELETE -- opTime:数据库中SQL的执行时间 -- ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下: -
- -```java -long id = Long.parseLong("6815665753853923328"); - long res = id >> 22; - DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - System.out.println(sdf.format(res)); //2021-06-29 23:42:24 -``` - -
- -## 八、数据类型 -| 支持 | NUMBER、SMALLINT、INT INTEGER、FLOAT、DECIMAL、NUMERIC、BINARY_FLOAT、BINARY_DOUBLE | -| -- | -- | -| | CHAR、NCHAR、NVARCHAR2、ROWID、VARCHAR2、VARCHAR、LONG、RAW、LONG RAW、INTERVAL YEAR、INTERVAL DAY、BLOB、CLOB、NCLOB | -| | DATE、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZONE | -| 暂不支持 | BFILE、XMLTYPE、Collections | - - -
- - -## 九、脚本示例 -见项目内`ChunJun : Local : Test`模块中的`demo`文件夹。 diff --git "a/docs/connectors/logminer/LogMiner\351\205\215\347\275\256.md" "b/docs/connectors/logminer/LogMiner\351\205\215\347\275\256.md" deleted file mode 100644 index 519c7fe9ee..0000000000 --- "a/docs/connectors/logminer/LogMiner\351\205\215\347\275\256.md" +++ /dev/null @@ -1,434 +0,0 @@ -# Oracle配置LogMiner - -目录: - - -- [Oracle配置LogMiner](#oracle配置logminer) - - [一、Oracle 10g(单机版)](#一oracle-10g单机版) - - [1、查询Oracle版本信息,这里配置的是`Oracle 10g`](#1查询oracle版本信息这里配置的是oracle-10g) - - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档) - - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意) - - [a、配置归档日志保存的路径](#a配置归档日志保存的路径) - - [b、关闭数据库](#b关闭数据库) - - [c、开启日志归档](#c开启日志归档) - - [d、开启扩充日志](#d开启扩充日志) - - [e、开启数据库](#e开启数据库) - - [4、配置日志组](#4配置日志组) - - [a、查询默认日志组信息](#a查询默认日志组信息) - - [b、查询日志组储存路径](#b查询日志组储存路径) - - [c、新增日志组与删除原有日志组](#c新增日志组与删除原有日志组) - - [d、查询创建的日志组](#d查询创建的日志组) - - [5、检查是否安装LogMiner工具](#5检查是否安装logminer工具) - - [6、创建LogMiner角色并赋权](#6创建logminer角色并赋权) - - [7、创建LogMiner用户并赋权](#7创建logminer用户并赋权) - - [8、验证用户权限](#8验证用户权限) - - [二、Oracle 11g(单机版)](#二oracle-11g单机版) - - [1、查询Oracle版本信息,这里配置的是`Oracle 11g`](#1查询oracle版本信息这里配置的是oracle-11g) - - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档-1) - - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意-1) - - [a、配置归档日志保存的路径](#a配置归档日志保存的路径-1) - - [b、关闭数据库](#b关闭数据库-1) - - [c、开启日志归档](#c开启日志归档-1) - - [d、开启扩充日志](#d开启扩充日志-1) - - [e、开启数据库](#e开启数据库-1) - - [4、检查是否安装LogMiner工具](#4检查是否安装logminer工具) - - [5、创建LogMiner角色并赋权](#5创建logminer角色并赋权) - - [6、创建LogMiner用户并赋权](#6创建logminer用户并赋权) - - [7、验证用户权限](#7验证用户权限) - - [三、Oracle 12c(单机版非CBD)](#三oracle-12c单机版非cbd) - - [1、查询Oracle版本信息,这里配置的是`Oracle 12c`](#1查询oracle版本信息这里配置的是oracle-12c) - - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档-2) - - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意-2) - - [a、配置归档日志保存的路径](#a配置归档日志保存的路径-2) - - [b、关闭数据库](#b关闭数据库-2) - - [c、开启日志归档](#c开启日志归档-2) - - [d、开启扩充日志](#d开启扩充日志-2) - - [e、开启数据库](#e开启数据库-2) - - [4、创建LogMiner角色并赋权](#4创建logminer角色并赋权) - - [5、创建LogMiner用户并赋权](#5创建logminer用户并赋权) - - [6、验证用户权限](#6验证用户权限) - - - -注意: - -1、某个Oracle数据源能同时运行的任务数量取决于该Oracle的内存大小 - -2、若数据量太大导致日志组频繁切换需要增加日志组数量,增大单个日志组存储大小 - -## 一、Oracle 10g(单机版) -### 1、查询Oracle版本信息,这里配置的是`Oracle 10g` -```sql ---查看oracle版本 -select * from v$version; -``` -
- -
-本章Oracle的版本如上图所示。 - - -### 2、通过命令行方式登录Oracle,查看是否开启日志归档 -```sql ---查询数据库归档模式 -archive log list; -``` -
- -
-图中显示`No Archive Mode`表示未开启日志归档。 - - -### 3、开启日志归档,开启日志归档需要重启数据库,请注意 -#### a、配置归档日志保存的路径 -根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 -```shell -# 创建归档日志保存目录 -mkdir -p /data/oracle/archivelog - -# 进入Oracle目录 -cd $ORACLE_HOME - -# 查看Oracle权限组,本章权限组如下图所示 -ls -l - -# 对归档日志保存目录赋予相应权限 -chown -R 下图中的用户名:下图中的组名 /data/oracle/ -``` -
- -
- -```sql ---配置归档日志保存的路径 -alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; -``` -#### b、关闭数据库 -```sql -shutdown immediate; -startup mount; -``` -#### c、开启日志归档 -```sql ---开启日志归档 -alter database archivelog; -``` -#### d、开启扩充日志 -```sql ---开启扩充日志 -alter database add supplemental log data (all) columns; -``` -#### e、开启数据库 -```sql -alter database open; -``` -再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 -
- -
- -### 4、配置日志组 -#### a、查询默认日志组信息 -```sql -SELECT * FROM v$log; -``` -
- -
- -如上图所示,日志组的默认数量为2组,大小为4194304/1024/1024 = 4MB,这意味着日志大小每达到4MB就会进行日志组的切换,切换太过频繁会导致查询出错,因此需要增加日志组数量及大小。 -#### b、查询日志组储存路径 -```sql -SELECT * FROM v$logfile; -``` -
- -
- -如上图所示,默认路径为`/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/`。 -#### c、新增日志组与删除原有日志组 -请与DBA联系,决定是否可以删除原有日志组。 -```sql ---增加两组日志组 -alter database add logfile group 3 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo3.log') size 200m; -alter database add logfile group 4 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo4.log') size 200m; -``` -```sql ---删除原有两组日志组,并继续新增两组日志组 -alter system checkpoint; -alter system switch logfile; -alter database drop logfile group 1; -alter database drop logfile group 2; -alter database add logfile group 1 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo1.log') size 200m; -alter database add logfile group 2 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo2.log') size 200m; -``` -#### d、查询创建的日志组 -```sql -SELECT * FROM v$log; -SELECT * FROM v$logfile; -``` -
- -
- -
- -
- -### 5、检查是否安装LogMiner工具 -Oracle10g默认已安装LogMiner工具包,通过以下命令查询: -```sql -desc DBMS_LOGMNR; -desc DBMS_LOGMNR_D; -``` -若无信息打印,则执行下列SQL初始化LogMiner工具包: -```sql -@$ORACLE_HOME/rdbms/admin/dbmslm.sql; -@$ORACLE_HOME/rdbms/admin/dbmslmd.sql; -``` - - -### 6、创建LogMiner角色并赋权 -其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 -```sql -create role roma_logminer_privs; -grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; -grant select_catalog_role to roma_logminer_privs; -``` - - -### 7、创建LogMiner用户并赋权 -其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 -```sql -create user roma_logminer identified by password default tablespace users; -grant roma_logminer_privs to roma_logminer; -grant execute_catalog_role to roma_logminer; -alter user roma_logminer quota unlimited on users; -``` - - -### 8、验证用户权限 -以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: -```sql - SELECT * FROM USER_ROLE_PRIVS; -``` -
- -
- -```sql -SELECT * FROM SESSION_PRIVS; -``` -
- -
- -至此,Oracle 10g数据库LogMiner实时采集配置完毕。 - - -## 二、Oracle 11g(单机版) -### 1、查询Oracle版本信息,这里配置的是`Oracle 11g` -```sql ---查看oracle版本 -select * from v$version; -``` -
- -
-本章Oracle的版本如上图所示。 - - -### 2、通过命令行方式登录Oracle,查看是否开启日志归档 -```sql ---查询数据库归档模式 -archive log list; -``` -
- -
-图中显示`No Archive Mode`表示未开启日志归档。 - - -### 3、开启日志归档,开启日志归档需要重启数据库,请注意 -#### a、配置归档日志保存的路径 -根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 -```sql - alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; -``` -#### b、关闭数据库 -```sql -shutdown immediate; -startup mount; -``` -#### c、开启日志归档 -```sql ---开启日志归档 -alter database archivelog; -``` -#### d、开启扩充日志 -```sql ---开启扩充日志 -alter database add supplemental log data (all) columns; -``` -#### e、开启数据库 -```sql -alter database open; -``` -再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 -
- -
- -### 4、检查是否安装LogMiner工具 -Oracle11g默认已安装LogMiner工具包,通过以下命令查询: -```sql -desc DBMS_LOGMNR; -desc DBMS_LOGMNR_D; -``` -若无信息打印,则执行下列SQL初始化LogMiner工具包: -```sql -@$ORACLE_HOME/rdbms/admin/dbmslm.sql; -@$ORACLE_HOME/rdbms/admin/dbmslmd.sql; -``` - - -### 5、创建LogMiner角色并赋权 -其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 -```sql -create role roma_logminer_privs; -grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; -grant select_catalog_role to roma_logminer_privs; -``` - - -### 6、创建LogMiner用户并赋权 -其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 -```sql -create user roma_logminer identified by password default tablespace users; -grant roma_logminer_privs to roma_logminer; -grant execute_catalog_role to roma_logminer; -alter user roma_logminer quota unlimited on users; -``` - - -### 7、验证用户权限 -以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: -```sql - SELECT * FROM USER_ROLE_PRIVS; -``` -
- -
- -```sql -SELECT * FROM SESSION_PRIVS; -``` -
- -
- -至此,Oracle 11g数据库LogMiner实时采集配置完毕。 - - -## 三、Oracle 12c(单机版非CBD) -### 1、查询Oracle版本信息,这里配置的是`Oracle 12c` -```sql ---查看oracle版本 -select BANNER from v$version; -``` -
- -
-本章Oracle的版本如上图所示。 - - -### 2、通过命令行方式登录Oracle,查看是否开启日志归档 -```sql ---查询数据库归档模式 -archive log list; -``` -
- -
-图中显示`No Archive Mode`表示未开启日志归档。 - - -### 3、开启日志归档,开启日志归档需要重启数据库,请注意 -#### a、配置归档日志保存的路径 -根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 -```sql - alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; -``` -#### b、关闭数据库 -```sql -shutdown immediate; -startup mount; -``` -#### c、开启日志归档 -```sql ---开启日志归档 -alter database archivelog; -``` -#### d、开启扩充日志 -```sql ---开启扩充日志 -alter database add supplemental log data (all) columns; -``` -#### e、开启数据库 -```sql -alter database open; -``` -再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 -
- -
- -### 4、创建LogMiner角色并赋权 -其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 -```sql -create role roma_logminer_privs; -grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,logmining,select any dictionary to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; -grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; -grant select_catalog_role to roma_logminer_privs; -grant LOGMINING to roma_logminer_privs; -``` - - -### 5、创建LogMiner用户并赋权 -其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 -```sql -create user roma_logminer identified by password default tablespace users; -grant roma_logminer_privs to roma_logminer; -grant execute_catalog_role to roma_logminer; -alter user roma_logminer quota unlimited on users; -``` - - -### 6、验证用户权限 -以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: -```sql - SELECT * FROM USER_ROLE_PRIVS; -``` -
- -
- -```sql -SELECT * FROM SESSION_PRIVS; -``` -
- -
- -至此,Oracle 12c数据库LogMiner实时采集配置完毕。 diff --git a/docs/connectors/mysql/mysql-sink.md b/docs/connectors/mysql/mysql-sink.md deleted file mode 100644 index 317729ebf3..0000000000 --- a/docs/connectors/mysql/mysql-sink.md +++ /dev/null @@ -1,214 +0,0 @@ -# Mysql Sink - -## 一、介绍 -mysql sink - -## 二、支持版本 -mysql5.x - - -## 三、插件名称 -| Sync | mysqlsink、mysqlwriter | -| --- | --- | -| SQL | mysql-x | - - -## 四、参数说明 -### 1、Sync -- **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - - 必选:是 - - 参数类型:List - - 默认值:无 - ```text - "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **column** - - 描述:目的表需要写入数据的字段。例如: "column": [{"name":"id",type:"varchar"}] - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **fullColumn** - - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 - - 必选:否 - - 参数类型:List - - 默认值:无 -
- -- **preSql** - - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - - 必选:否 - - 参数类型:List - - 默认值:无 -
- -- **postSql** - - 描述:写入数据到目的表后,会执行这里的一组标准语句 - - 必选:否 - - 参数类型:List - - 默认值:无 -
- -- **mode** - - 描述:控制写入数据到目标表采用 insert into 或者 replace into 或者 ON DUPLICATE KEY UPDATE 语句 - - 必选:是 - - 所有选项:insert/replace/update - - 参数类型:String - - 默认值:insert -
- -- **batchSize** - - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 - - 必选:否 - - 参数类型:int - - 默认值:1 -
- -- **updateKey** - - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 - - 注意: - - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - - 必选:否 - - 参数类型:List - - 示例:"updateKey": ["id"] - - 默认值:无 -
- -- **semantic** - - 描述:sink端是否支持二阶段提交 - - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once - - 必选:否 - - 参数类型:String - - 示例:"semantic": "exactly-once" - - 默认值:at-least-once -
- -### 2、SQL -- **connector** - - 描述:mysql-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **url** - - 描述:jdbc:mysql://localhost:3306/test - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **sink.buffer-flush.max-rows** - - 描述:批量写数据条数,单位:条 - - 必选:否 - - 参数类型:String - - 默认值:1024 -
- -- **sink.buffer-flush.interval** - - 描述:批量写时间间隔,单位:毫秒 - - 必选:否 - - 参数类型:String - - 默认值:10000 -
- -- **sink.all-replace** - - 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null) - - 必选:否 - - 参数类型:String - - 默认值:false -
- -- **sink.parallelism** - - 描述:写入结果的并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **sink.semantic** - - 描述:sink端是否支持二阶段提交 - - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once - - 必选:否 - - 参数类型:String - - 示例:"semantic": "exactly-once" - - 默认值:at-least-once -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/mysql/mysql-source.md b/docs/connectors/mysql/mysql-source.md deleted file mode 100644 index 307b232021..0000000000 --- a/docs/connectors/mysql/mysql-source.md +++ /dev/null @@ -1,305 +0,0 @@ -# Mysql Source - -## 一、介绍 -支持从mysql离线读取,支持mysql实时间隔轮询读取 - -## 二、支持版本 -mysql5.x - - -## 三、插件名称 -| Sync | mysqlsource、mysqlreader | -| --- | --- | -| SQL | mysql-x | - - -## 四、参数说明 -### 1、Sync -- **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - - 必选:是 - - 参数类型:List - - 默认值:无 - ```text - "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **fetchSize** - - 描述:一次性从数据库中读取多少条数据,MySQL默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2、连接参数useCursorFetch=true。 -注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - - 必选:否 - - 参数类型:int - - 默认值:1024 -
- -- **where** - - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **splitPk** - - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - - 注意: - - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun将报错。 - - 如果channel大于1但是没有配置此参数,任务将置为失败。 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **queryTimeOut** - - 描述:查询超时时间,单位秒。 - - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - - 必选:否 - - 参数类型:int - - 默认值:1000 -
- -- **customSql** - - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 - - 注意: - - 只能是查询语句,否则会导致任务失败; - - 查询语句返回的字段需要和column列表里的字段对应; - - 当指定了此参数时,connection里指定的table无效; - - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **column** - - 描述:需要读取的字段。 - - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash - "column":["*"] - ``` - 2.只指定字段名称: - ``` - "column":["id","name"] - ``` - 3.指定具体信息: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - 必选:是 - - 默认值:无 -
- -- **polling** - - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的起始位置。 - - 必选:否 - - 参数类型:Boolean - - 默认值:false -
- -- **pollingInterval** - - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - - 必选:否 - - 参数类型:long - - 默认值:5000 -
- -- **increColumn** - - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) - - 必选:否 - - 参数类型:String或int - - 默认值:无 -
- -- **startLocation** - - 描述:增量查询起始位置 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **useMaxFunc** - - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true - - 必选:否 - - 参数类型:Boolean - - 默认值:false -
- -- **requestAccumulatorInterval** - - 描述:发送查询累加器请求的间隔时间。 - - 必选:否 - - 参数类型:int - - 默认值:2 -
- -### 2、SQL -- **connector** - - 描述:mysql-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **url** - - 描述:jdbc:mysql://localhost:3306/test - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **scan.polling-interval** - - 描述:间隔轮训时间。非必填(不填为离线任务),无默认 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.parallelism** - - 描述:并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.fetch-size** - - 描述:每次从数据库中fetch大小,单位:条。 - - 必选:否 - - 参数类型:String - - 默认值:1024 -
- -- **scan.query-timeout** - - 描述:数据库连接超时时间,单位:秒。 - - 必选:否 - - 参数类型:String - - 默认值:1 -
- -- **scan.partition.column** - - 描述:多并行度读取的切分字段 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.partition.strategy** - - 描述:数据分片策略 - - 必选:否 - - 参数类型:String - - 默认值:range -
- -- **scan.increment.column** - - 描述:增量字段名称 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.increment.column-type** - - 描述:增量字段类型 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.start-location** - - 描述:增量字段开始位置,如果不指定则先同步所有,然后在增量 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.restore.columnname** - - 描述:开启了cp,任务从sp/cp续跑字段名称。如果续跑,则会覆盖scan.start-location开始位置,从续跑点开始 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.restore.columntype** - - 描述:开启了cp,任务从sp/cp续跑字段类型 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/oracle/oracle-lookup.md b/docs/connectors/oracle/oracle-lookup.md deleted file mode 100644 index 7e5809ccfe..0000000000 --- a/docs/connectors/oracle/oracle-lookup.md +++ /dev/null @@ -1,104 +0,0 @@ -# Oracle Lookup - -## 一、介绍 -Oracle维表,支持全量和异步方式
-全量缓存:将维表数据全部加载到内存中,建议数据量不大使用。
-异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 - -## 二、支持版本 -Oracle 9 及以上 - - -## 三、插件名称 -| SQL | oracle-x | -| --- | --- | - -## 四、参数说明 -- **connector** - - 描述:oracle-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **url** - - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **lookup.cache-type** - - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - - 必选:否 - - 参数类型:string - - 默认值:LRU -
- -- **lookup.cache-period** - - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - - 必选:否 - - 参数类型:string - - 默认值:3600000 -
- -- **lookup.cache.max-rows** - - 描述:lru维表缓存数据的条数,默认10000条 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **lookup.cache.ttl** - - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - - 必选:否 - - 参数类型:string - - 默认值:60000 -
- -- **lookup.fetch-size** - - 描述:ALL维表每次从数据库加载的条数,默认1000条 - - 必选:否 - - 参数类型:string - - 默认值:1000 -
- -- **lookup.parallelism** - - 描述:维表并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -## 五、数据类型 -| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| 暂不支持 | BFILE、XMLTYPE、Collections、BLOB、CLOB、NCLOB | - -注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 - - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/oracle/oracle-source.md b/docs/connectors/oracle/oracle-source.md deleted file mode 100644 index 408f8cb420..0000000000 --- a/docs/connectors/oracle/oracle-source.md +++ /dev/null @@ -1,307 +0,0 @@ -# Oracle Source - -## 一、介绍 -支持从oracle离线读取 - -## 二、支持版本 -Oracle 9 及以上 - - -## 三、插件名称 -| Sync | oraclesource、oraclereader | -| --- | --- | -| SQL | oracle-x | - - -## 四、参数说明 -### 1、Sync -- **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - - 必选:是 - - 参数类型:List - - 默认值:无 - ```text - "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) - - 必选:是 - - 参数类型:string - - 默认值:用户名 -
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **fetchSize** - - 描述:一次性从数据库中读取多少条数据,ORACLE默认fetchSize大小为10。当fetchSize设置过小时导致频繁读取数据会影响查询速度,以及数据库压力。当fetchSize设置过大时在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - - 必选:否 - - 参数类型:int - - 默认值:1024 -
- -- **where** - - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **splitPk** - - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - - 注意: - - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun将报错。 - - 如果channel大于1但是没有配置此参数,任务将置为失败。 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **queryTimeOut** - - 描述:查询超时时间,单位秒。 - - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - - 必选:否 - - 参数类型:int - - 默认值:1000 -
- -- **customSql** - - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 - - 注意: - - 只能是查询语句,否则会导致任务失败; - - 查询语句返回的字段需要和column列表里的字段对应; - - 当指定了此参数时,connection里指定的table无效; - - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **column** - - 描述:需要读取的字段。 - - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash - "column":["*"] - ``` - 2.只指定字段名称: - ``` - "column":["id","name"] - ``` - 3.指定具体信息: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - 必选:是 - - 默认值:无 -
- -- **polling** - - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的起始位置。 - - 必选:否 - - 参数类型:Boolean - - 默认值:false -
- -- **pollingInterval** - - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - - 必选:否 - - 参数类型:long - - 默认值:5000 -
- -- **increColumn** - - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) - - 必选:否 - - 参数类型:String或int - - 默认值:无 -
- -- **startLocation** - - 描述:增量查询起始位置 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **useMaxFunc** - - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true - - 必选:否 - - 参数类型:Boolean - - 默认值:false -
- -- **requestAccumulatorInterval** - - 描述:发送查询累加器请求的间隔时间。 - - 必选:否 - - 参数类型:int - - 默认值:2 -
- -### 2、SQL -- **connector** - - 描述:oracle-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **url** - - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **scan.polling-interval** - - 描述:间隔轮训时间。非必填(不填为离线任务),无默认 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.parallelism** - - 描述:并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.fetch-size** - - 描述:每次从数据库中fetch大小,单位:条。 - - 必选:否 - - 参数类型:String - - 默认值:1024 -
- -- **scan.query-timeout** - - 描述:数据库连接超时时间,单位:秒。 - - 必选:否 - - 参数类型:String - - 默认值:1 -
- -- **scan.partition.column** - - 描述:多并行度读取的切分字段,多并行度下必需要设置 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.partition.strategy** - - 描述:数据分片策略 - - 必选:否 - - 参数类型:String - - 默认值:range -
- -- **scan.increment.column** - - 描述:增量字段名称 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.increment.column-type** - - 描述:增量字段类型 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.start-location** - - 描述:增量字段开始位置,如果不指定则先同步所有,然后在增量 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.restore.columnname** - - 描述:开启了cp,任务从sp/cp续跑字段名称。如果续跑,则会覆盖scan.start-location开始位置,从续跑点开始 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **scan.restore.columntype** - - 描述:开启了cp,任务从sp/cp续跑字段类型 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -## 五、数据类型 -| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| 暂不支持 | BFILE、XMLTYPE、Collections | -| 仅在 Sync 中支持 | BLOB、CLOB、NCLOB | - -注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/redis/redis-lookup.md b/docs/connectors/redis/redis-lookup.md deleted file mode 100644 index 21cb4b0fb6..0000000000 --- a/docs/connectors/redis/redis-lookup.md +++ /dev/null @@ -1,136 +0,0 @@ -# Redis Lookup - -## 一、介绍 -redis维表,支持全量和异步方式
-全量缓存:将维表数据全部加载到内存中,建议数据量不大使用。
-异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 - -## 二、支持版本 -主流版本 - - -## 三、插件名称 -| SQL | redis-x | -| --- | --- | - -## 四、参数说明 -- **connector** - - 描述:redis-x - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **url** - - 描述:localhost:6379 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **table-name** - - 描述:tableName - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **redis-type** - - 描述: redis模式(1 单机,2 哨兵, 3 集群),默认:1 - - 必选:否 - - 参数类型:string - - 默认值:1 -
- -- **master-name** - - 描述: 主节点名称(哨兵模式下为必填项) - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **database** - - 描述: redis 的数据库地址,默认:0 - - 必选:否 - - 参数类型:string - - 默认值:0 -
- -- **timeout** - - 描述:连接超时时间,默认:10000毫秒 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **max.total** - - 描述:最大连接数 ,默认:8 - - 必选:否 - - 参数类型:string - - 默认值:8 -
- -- **max.idle** - - 描述:最大空闲连接数,默认:8 - - 必选:否 - - 参数类型:string - - 默认值:8 -
- -- **min.idle** - - 描述: 最小空闲连接数 ,默认:0 - - 必选:否 - - 参数类型:string - - 默认值:0 -
- -- **lookup.cache-type** - - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - - 必选:否 - - 参数类型:string - - 默认值:LRU -
- -- **lookup.cache-period** - - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - - 必选:否 - - 参数类型:string - - 默认值:3600000 -
- -- **lookup.cache.max-rows** - - 描述:lru维表缓存数据的条数,默认10000条 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **lookup.cache.ttl** - - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - - 必选:否 - - 参数类型:string - - 默认值:60000 -
- -- **lookup.parallelism** - - 描述:维表并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/redis/redis-sink.md b/docs/connectors/redis/redis-sink.md deleted file mode 100644 index 80bf173311..0000000000 --- a/docs/connectors/redis/redis-sink.md +++ /dev/null @@ -1,207 +0,0 @@ -# Redis Sink - -## 一、介绍 -redis sink - -## 二、支持版本 -主流版本 - - -## 三、插件名称 -| Sync | redissink、rediswriter | -| --- | --- | -| SQL | redis-x | - - -## 四、参数说明 -### 1、Sync -- **hostPort** - - 描述:Redis的IP地址和端口 - - 必选:是 - - 默认值:localhost:6379 -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 默认值:无 -
- -- **database** - - 描述:要写入Redis数据库 - - 必选:否 - - 默认值:0 -
- -- **keyFieldDelimiter** - - 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 - - 必选:否 - - 默认值:\u0001 -
- -- **dateFormat** - - 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” - - 必选:否 - - 默认值:将日期以long类型写入 -
- -- **expireTime** - - 描述:Redis value 值缓存失效时间(如果需要永久有效则可以不填该配置项)。 - - 注意:如果过期时间的秒数大于 60_60_24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 - - 必选:否 - - 默认值:0(0 表示永久有效) -
- -- **timeout** - - 描述:写入 Redis 的超时时间。 - - 单位:毫秒 - - 必选:否 - - 默认值:30000 -
- -- **type和mode** - - 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 - - 选项:string/list/set/zset/hash - - | type | 描述 | mode | 说明 | 注意 | - | ---- | ---- | ---- | ---- | ---- | - | string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | - | list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | - | list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | - | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | - | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | - | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | - - 必选:是 - - 默认值:无 -
- -- **valueFieldDelimiter** - - 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 - - 必选:否 - - 默认值:\u0001 -
- -- **keyIndexes** - - 描述:keyIndexes 表示源端哪几列需要作为 key(第一列是从 0 开始)。如果是第一列和第二列需要组合作为 key,那么 keyIndexes 的值则为 [0,1]。 - - 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。例如:Redis中的数据为 "test,redis,First,Second",keyIndexes = [0,1] ,因此得到的key为 "test\\u0001redis", value为 "First\\u0001Second" - - 必选:是 - - 默认值:无 -
- -### 2、SQL -- **connector** - - 描述:redis-x - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **url** - - 描述:localhost:6379 - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **table-name** - - 描述:tableName - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **redis-type** - - 描述: redis模式(1 单机,2 哨兵, 3 集群),默认:1 - - 必选:否 - - 参数类型:string - - 默认值:1 -
- -- **master-name** - - 描述: 主节点名称(哨兵模式下为必填项) - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **database** - - 描述: redis 的数据库地址,默认:0 - - 必选:否 - - 参数类型:string - - 默认值:0 -
- -- **timeout** - - 描述:连接超时时间,默认:10000毫秒 - - 必选:否 - - 参数类型:string - - 默认值:10000 -
- -- **max.total** - - 描述:最大连接数 ,默认:8 - - 必选:否 - - 参数类型:string - - 默认值:8 -
- -- **max.idle** - - 描述:最大空闲连接数,默认:8 - - 必选:否 - - 参数类型:string - - 默认值:8 -
- -- **min.idle** - - 描述: 最小空闲连接数 ,默认:0 - - 必选:否 - - 参数类型:string - - 默认值:0 -
- -- **key.expired-time** - - 描述:redis sink的key的过期时间。默认是0(永不过期),单位是s。默认:0 - - 必选:否 - - 参数类型:string - - 默认值:0 -
- -- **sink.parallelism** - - 描述:sink并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **type和mode** - - 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 - - 选项:string/list/set/zset/hash - - | type | 描述 | mode | 说明 | 注意 | - | ---- | ---- | ---- | ---- | ---- | - | string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | - | list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | - | list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | - | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | - | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | - | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | - - 必选:是 - - 默认值:无 -
- - -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/solr/solr-sink.md b/docs/connectors/solr/solr-sink.md deleted file mode 100644 index a3f6626362..0000000000 --- a/docs/connectors/solr/solr-sink.md +++ /dev/null @@ -1,146 +0,0 @@ -## 一、介绍 -Solr写入,目前只支持开启Kerberos的Solr数据源。 - - -## 二、支持版本 -Solr 7.4 - - -## 三、插件名称 -| Sync | solrsink、solrwriter | -| --- | --- | -| SQL | solr-x | - - -## 四、参数说明 -### 1、Sync - -- **zkHosts** - - 描述:Solr的zookeeper集群地址,每个ZK节点为数组的一个元素 - - 必选:是 - - 参数类型:array - - 默认值:无 - - -- **zkChroot** - - 描述:Solr所在的Zookeeper chroot - - 必选:否 - - 参数类型:string - - 默认值:无 - - -- **collection** - - 描述:Solr collection名称 - - 必选:是 - - 参数类型:string - - 默认值:无 - - -- **column** - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: -```json -"column": [ - { - "name": "val_int", - "type": "long" - } -] -``` - -- 必选:是 -- 参数类型:数组 -- 默认值:无 - -- **kerberosConfig** - - 描述:开启kerberos时包含kerberos相关配置 - - 必选:否 - - 格式: -```json -"kerberosConfig": { - "principal": "solr/worker@DTSTACK.COM", - "keytab":"./solr.keytab", - "krb5conf":"./krb5.conf" -} -``` - -- 参数类型:object -- 默认值:无 -- **batchSize** - - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 - - 必选:否 - - 字段类型:int - - 默认值:1 -- **flushIntervalMills** - - 描述:批量写入时间间隔:单位毫秒。 - - 必选:否 - - 字段类型:int - - 默认值:10000 - -### 2、SQL - -- **zk-hosts** - - 描述:Solr的zookeeper集群地址,每个节点用, 分割 - - 必选:是 - - 参数类型:string - - 默认值:无 -- **zk-chroot** - - 描述:Solr所在的Zookeeper chroot - - 必选:否 - - 参数类型:string - - 默认值:无 -- **collection** - - 描述:Solr collection名称 - - 必选:是 - - 参数类型:string - - 默认值:无 - - - -- **security.kerberos.principal** - - 描述:kerberos认证的principal - - 必选:是 - - 默认值:无 -- **security.kerberos.keytab** - - 描述:kerberos认证的keytab文件路径 - - 必选:是 - - 默认值:无 -- **security.kerberos.krb5conf** - - 描述:kerberos认证的krb5conf文件路径 - - 必选:是 - - 默认值:无 - - - -- **sink.parallelism** - - 描述:sink并行度 - - 必选:是 - - 默认值:无 -- **sink.buffer-flush.max-rows** - - 描述:批量写入条数 - - 必选:否 - - 默认值:无 -- **sink.buffer-flush.interval** - - 描述:批量写入时间间隔:单位毫秒。 - - 必选:否 - - 默认值:无 - - - -## 五、数据类型 -| 支持 | bool | -| --- | --- | -| | int | -| | long | -| | string | -| | text | -| | float | -| | double | -| | date | -| 暂不支持 | array | - - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/solr/solr-source.md b/docs/connectors/solr/solr-source.md deleted file mode 100644 index 50e625482f..0000000000 --- a/docs/connectors/solr/solr-source.md +++ /dev/null @@ -1,90 +0,0 @@ -## 一、介绍 -Solr读取,目前只支持开启Kerberos的Solr数据源。暂不支持SQL读。 - -## 二、支持版本 -Solr 7.4 - -## 三、插件名称 -| Sync | solrsource、solrreader | -| --- | --- | - - - -## 四、参数说明 -### 1、Sync - -- **zkHosts** - - 描述:Solr的zookeeper集群地址,每个ZK节点为数组的一个元素。 - - 必选:是 - - 参数类型:array - - 默认值:无 - - - -- **zkChroot** - - 描述:Solr所在的Zookeeper chroot - - 必选:否 - - 参数类型:string - - 默认值:无 - - - -- **collection** - - 描述:Solr collection名称 - - 必选:是 - - 参数类型:string - - 默认值:无 - - - -- **column** - - 描述:需要读取的字段 - - 注意:不支持*格式 - - 格式: -```json -"column": [ - { - "name": "val_int", - "type": "long" - } -] -``` - -- 必选:是 -- 参数类型:数组 -- 默认值:无 - - - -- **kerberosConfig** - - 描述:开启kerberos时包含kerberos相关配置 - - 必选:否 - - 格式: -```json -"kerberosConfig": { - "principal": "solr/worker@DTSTACK.COM", - "keytab":"./solr.keytab", - "krb5conf":"./krb5.conf" -} -``` - -- 参数类型:object -- 默认值:无 - - - -## 五、数据类型 -| 支持 | bool | -| --- | --- | -| | int | -| | long | -| | string | -| | text | -| | float | -| | double | -| | date | -| 暂不支持 | array | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/sqlservercdc/SqlserverCDC-source.md b/docs/connectors/sqlservercdc/SqlserverCDC-source.md deleted file mode 100644 index 9d67ce5242..0000000000 --- a/docs/connectors/sqlservercdc/SqlserverCDC-source.md +++ /dev/null @@ -1,259 +0,0 @@ -# SqlserverCDC Source - - - -- [一、介绍](#一介绍) -- [二、支持版本](#二支持版本) -- [三、数据库配置](#三数据库配置) -- [四、SqlserverCDC原理](#四SqlserverCDC原理) -- [五、插件名称](#五插件名称) -- [六、参数说明](#六参数说明) - - [1、Sync](#1sync) - - [2、SQL](#2sql) -- [七、数据结构](#七数据结构) -- [八、数据类型](#八数据类型) -- [九、脚本示例](#九脚本示例) - - - -
- -## 一、介绍 -Sqlservercdc插件支持配置监听表名称以及读取起点读取日志数据。SQLservercdc在checkpoint时保存当前消费的lsn,因此支持续跑。 -
- -## 二、支持版本 -SqlServer 2012、2014、2016、2017、2019单机版 -
- -## 三、数据库配置 -[SqlserverCDC配置](SqlserverCDC配置.md) -
- -## 四、SqlserverCDC原理 -[SqlserverCDC原理](SqlserverCDC原理.md) -
- -## 五、插件名称 -| Sync | sqlservercdcreader、sqlservercdcsource | -| --- | --- | -| SQL | sqlservercdc-x | - -
- -## -## 六、参数说明 -### 1、Sync - -- **url** - - 描述:sqlserver数据库的JDBC URL链接 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- - -- **username** - - 描述:用户名 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **password** - - 描述:密码 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **tableList** - - 描述: 需要监听的表,如:["schema1.table1","schema1.table2"] - - 必选:是 - - 字段类型:数组 - - 默认值:无 - -
- - -- **splitUpdate** - - 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见【六、数据结构说明】 - - 必选:否 - - 字段类型:boolean - - 默认值:false - -
- -- **cat** - - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 - - 必选:否 - - 字段类型:String - - 默认值:UPDATE,INSERT,DELETE - -
- -- **lsn** - - 描述: 要读取SqlServer CDC日志序列号的开始位置 - - 必选: 否 - - 字段类型:String(00000032:00002038:0005) - - 默认值:无 - -
- -- **pollInterval** - - 描述: 监听拉取SqlServer CDC数据库间隔时间,该值越小,采集延迟时间越小,给数据库的访问压力越大 - - 必选:否 - - 字段类型:long(单位毫秒) - - 默认值:1000 - -
- - -- **pavingData** - - 描述:是否将解析出的json数据拍平,具体见【七、数据结构说明】 - - 必选:否 - - 字段类型:boolean - - 默认值:false - -
- -### 2、SQL -- **url** - - 描述:sqlserver数据库的JDBC URL链接 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **username** - - 描述:用户名 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **password** - - 描述:密码 - - 必选:是 - - 参数类型:string - - 默认值:无 - -
- -- **table** - - 描述:需要解析的数据表。 - - 注意:SQL任务只支持监听单张表,且数据格式为schema.table - - 必选:否 - - 字段类型:string - - 默认值:无 - -
- -- **cat** - - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 - - 必选:否 - - 字段类型:String - - 默认值:UPDATE,INSERT,DELETE - -
- -- **lsn** - - 描述: 要读取SqlServer CDC日志序列号的开始位置 - - 必选: 否 - - 字段类型:String(00000032:00002038:0005) - - 默认值:无 - -
- -- **poll-interval** - - 描述: 监听拉取SqlServer CDC数据库间隔时间,该值越小,采集延迟时间越小,给数据库的访问压力越大 - - 必选:否 - - 字段类型:long(单位毫秒) - - 默认值:1000 - -
- -## 七、数据结构 -在2020-01-01 12:30:00(时间戳:1577853000000)执行: -```sql -INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) VALUES (1, 1, 'a'); -``` -在2020-01-01 12:31:00(时间戳:1577853060000)执行: -```sql -DELETE FROM `tudou`.`kudu` WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; -``` -在2020-01-01 12:32:00(时间戳:1577853180000)执行: -```sql -UPDATE `tudou`.`kudu` SET `id` = 2, `user_id` = 2, `name` = 'b' WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; -``` -1、pavingData = true, splitUpdate = false -RowData中的数据依次为: -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", 1, 1, "a", 2, 2, "b"] -``` -2、pavingData = false, splitUpdate = false -RowData中的数据依次为: -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", {"id":1, "user_id":1, "name":"a"}, {"id":2, "user_id":2, "name":"b"}] -``` -3、pavingData = true, splitUpdate = true -RowData中的数据依次为: -``` -//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] - -//schema, table, ts, opTime, type, before_id, before_user_id, before_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", 1, 1, "a"] - -//schema, table, ts, opTime, type, after_id, after_user_id, after_name -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", 2, 2, "b"] -``` -4、pavingData = false, splitUpdate = true -RowData中的数据依次为: -``` -//schema, table, ts, opTime, type, before, after -["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] -["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] -//schema, table, ts, opTime, type, before -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", {"id":1, "user_id":1, "name":"a"}] -//schema, table, ts, opTime, type, after -["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", {"id":2, "user_id":2, "name":"b"}] -``` - -- type:变更类型,INSERT,UPDATE、DELETE -- opTime:数据库中SQL的执行时间 -- ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下: -```java -long id = Long.parseLong("6760525407742726144"); -long res = id >> 22; -DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -System.out.println(sdf.format(res)); //2021-01-28 19:54:21 -``` - -## 八、数据类型 -| 支持 | BIT | -| --- | --- | -| | TINYINT24、INT、INTEGER、FLOAT、DOUBLE、REAL、LONG、BIGINT、DECIMAL、NUMERIC | -| | CHAR、VARCHAR、NCHAR、NVARCHAR、TEXT | -| | DATE、TIME、TIMESTAMP、DATETIME、DATETIME2、SMALLDATETIME | -| | BINARY、VARBINARY | -| 暂不支持 | ROWVERSION、UNIQUEIDENTIFIER、CURSOR、TABLE、SQL_VARIANT | - - -
- - -## 九、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/starrocks/starrocks-sink.md b/docs/connectors/starrocks/starrocks-sink.md deleted file mode 100644 index ae4eec4c7f..0000000000 --- a/docs/connectors/starrocks/starrocks-sink.md +++ /dev/null @@ -1,225 +0,0 @@ -# StarRocks Sink - -## 一、介绍 -StarRocks Sink插件支持向StarRocks数据库以Stream load方式写入数据 - -## 二、支持版本 -StarRocks 1.19 + - - -## 三、插件名称 -| Sync | starrockssink、starrockswriter | -| --- | --- | -| SQL | starrocks-x | - - -## 四、参数说明 -### 1、Sync -- **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - - 必选:是 - - 参数类型:List - - 默认值:无 - ```text - "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], - "table": ["table"], - "schema":"public" - }] - ``` -
- -- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - 必选:是 - - 参数类型:string - - 默认值:无 -
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 参数类型:string - - 默认值:无 -
- -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **column** - - 描述:目的表需要写入数据的字段。例如: "column": [{"name":"id",type:"varchar"}] - - 必选:是 - - 参数类型:List - - 默认值:无 -
- -- **fullColumn** - - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 - - 必选:否 - - 参数类型:List - - 默认值:无 -
- -- **preSql** - - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - - 必选:否 - - 参数类型:List - - 默认值:无 -
- -- **loadUrl** - - 描述:FE地址, 例如:fe_ip:http_port, 多个地址分号连接 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **primaryKey** - - 描述:当向带有主键模型的表写入时,需要指定此参数的值 - - 必选:否 - - 参数类型:List - - 示例:"primaryKey": ["id"] - - 默认值:无 -
- -### 2、SQL -- **connector** - - 描述:starrocks-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **jdbc-url** - - 描述:jdbc:mysql://localhost:3306/test - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **load-url** - - 描述:FE地址,例如:localhost:8030 - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **database-name** - - 描述:库名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: -
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **sink.buffer-flush.max-rows** - - 描述:批量写数据条数,单位:条 - - 必选:否 - - 参数类型:String - - 默认值:1024 -
- -- **sink.buffer-flush.interval** - - 描述:批量写时间间隔,单位:毫秒 - - 必选:否 - - 参数类型:String - - 默认值:10000 -
- -- **sink.all-replace** - - 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null) - - 必选:否 - - 参数类型:String - - 默认值:false -
- -- **sink.parallelism** - - 描述:写入结果的并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **sink.semantic** - - 描述:sink端是否支持二阶段提交 - - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once - - 必选:否 - - 参数类型:String - - 示例:"semantic": "exactly-once" - - 默认值:at-least-once -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 - -## 七、写入相关统计指标为空 -当调用flink-connector-starrocks写入数据时,默认情况是不会更新chunjun里写入统计指标数据,需要修改flink-connector-starrocks的相关源码, -以 1.1.13_flink-1.12 版本为例,修改 StarRocksDynamicSinkFunction.java 的相关代码如下: - -``` -protected LongCounter bytesWriteCounter; -protected LongCounter durationCounter; -protected LongCounter numWriteCounter; - -bytesWriteCounter = getRuntimeContext().getLongCounter("byteWrite"); -durationCounter = getRuntimeContext().getLongCounter("writeDuration"); -numWriteCounter = getRuntimeContext().getLongCounter("numWrite"); - -long startTime = System.currentTimeMillis(); - -numWriteCounter.add(1); -durationCounter.add(System.currentTimeMillis() - startTime); -bytesWriteCounter.add(ObjectSizeCalculator.getObjectSize(value)); -``` - -具体代码位置如图: -
- -
- diff --git a/docs/connectors/stream/stream-sink.md b/docs/connectors/stream/stream-sink.md deleted file mode 100644 index e2e007c53e..0000000000 --- a/docs/connectors/stream/stream-sink.md +++ /dev/null @@ -1,47 +0,0 @@ -# Stream Sink - -## 一、介绍 -控制台显示数据,方便调试 - -## 二、支持版本 - - - -## 三、插件名称 -| Sync | streamsink、streamwriter | -| --- | --- | -| SQL | stream-x | - - -## 四、参数说明 -### 1、Sync -- **print** - - 描述:是否打印 - - 必选:否 - - 参数类型:boolean - - 默认值:是 -
- -### 2、SQL -- **connector** - - 描述:stream-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **print** - - 描述:是否打印 - - 必选:否 - - 参数类型:boolean - - 默认值:是 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/stream/stream-source.md b/docs/connectors/stream/stream-source.md deleted file mode 100644 index c0c2414d45..0000000000 --- a/docs/connectors/stream/stream-source.md +++ /dev/null @@ -1,103 +0,0 @@ -# Stream Source - -## 一、介绍 -为了让用户能快速熟悉与使用,ChunJun提供了不需要数据库就能读取数据的Stream reader插件。
该插件利用了模拟数据的JMockData框架,能够根据给定的属性生成相应的随机数据,方便用户修改和调试 - -## 二、支持版本 - - - -## 三、插件名称 -| Sync | streamsource、streamreader | -| --- | --- | -| SQL | stream-x | - - -## 四、参数说明 -### 1、Sync -- **sliceRecordCount** - - 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止 - - 必选:否 - - 参数类型:list - - 默认值:0 -
- -- **permitsPerSecond** - - 描述:限制每秒生产的条数,默认不限制 - - 必选:否 - - 参数类型:int - - 默认值:无 -
- -- **column** - - 描述:随机Java数据类型的字段信息 - - 格式:一组或多组描述"name"和"type"的json格式 - - 格式说明: - ```json - { - "name": "id", - "type": "int", - "value":"7" - } - ``` - - "name"属性为用户提供的标识,类似于mysql的列名,必须填写。 - - "tpye" 属性为需要生成的数据类型,可配置以下类型: - - id:从0开始步长为1的int类型自增ID - - int,integer - - byte - - boolean - - char,character - - short - - long - - float - - double - - date - - timestamp - - bigdecimal - - biginteger - - int[] - - byte[] - - boolean[] - - char[],character[] - - short[] - - long[] - - float[] - - double[] - - string[] - - binary - - string:以上均不匹配时默认为string字符串 - - "value"属性为用户设定的输出值,可以不填。 - - 参数类型:list - - 默认值:无 -
- -### 2、SQL -- **connector** - - 描述:stream-x - - 必选:是 - - 参数类型:String - - 默认值:无 -
- -- **number-of-rows** - - 描述:输入条数,默认无限 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -- **rows-per-second** - - 描述:每秒输入条数,默认不限制 - - 必选:否 - - 参数类型:String - - 默认值:无 -
- -## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | -| --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | - - -## 六、脚本示例 -见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/tidb/tidb-lookup.md b/docs/connectors/tidb/tidb-lookup.md deleted file mode 100644 index f66a9271b6..0000000000 --- a/docs/connectors/tidb/tidb-lookup.md +++ /dev/null @@ -1,125 +0,0 @@ -# TiDB Lookup - -## 一、介绍 -TiDB维表,支持全量和异步方式。
-全量缓存:将维表数据全部加载到内存中,建议数据量不大使用。
-异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。

- -## 二、支持版本 -TiDB 3.0.10之后、TiDB 4.0 - -## 三、插件名称 -| SQL | tidb-x | -| --- | --- | - -## 四、参数说明 - -- **connector** - - 描述:sqlserver-x - - 必选:是 - - 参数类型:String - - 默认值:无 - -
- -- **url** - - 描述:TiDB兼容MySQL JDBC连接驱动 - - 必选:是 - - 字段类型:String - - 默认值:无 - -
- -- **table-name** - - 描述:表名 - - 必选:是 - - 参数类型:String - - 默认值:无: - -
- -- **schema** - - 描述:schema - - 必选:否 - - 字段类型:String - - 默认值:无 - -​
- -- **username** - - 描述:username - - 必选:是 - - 参数类型:String - - 默认值:无 - -
- - -- **password** - - 描述:password - - 必选:是 - - 参数类型:String - - 默认值:无 - -
- - -- **lookup.cache-type** - - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - - 必选:否 - - 参数类型:string - - 默认值:LRU - -
- - -- **lookup.cache-period** - - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - - 必选:否 - - 参数类型:string - - 默认值:3600000 - -
- - -- **lookup.cache.max-rows** - - 描述:lru维表缓存数据的条数,默认10000条 - - 必选:否 - - 参数类型:string - - 默认值:10000 - -
- - -- **lookup.cache.ttl** - - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - - 必选:否 - - 参数类型:string - - 默认值:60000 - -
- - -- **lookup.fetch-size** - - 描述:ALL维表每次从数据库加载的条数,默认1000条 - - 必选:否 - - 参数类型:string - - 默认值:1000 - -
- - -- **lookup.parallelism** - - 描述:维表并行度 - - 必选:否 - - 参数类型:string - - 默认值:无 - - - -## 五、数据类型 -|支持 | BIT、BOOL、BOOLEAN、SMALLINT、MEDIUMINT、INT、INTEGER、BIGINT、FLOAT、DOUBLE、DECIMAL、DATE、TIME、DATETIME、TIMESTAMP、YEAR、CHAR、VARCHAR、TEXT、TINYTEXT、MEDIUMTEXT、LONGTEXT、BINARY、VARBINARY、BLOB、TINYBLOB、MEDIUMBLOB、LONGBLOB、ENUM、SET、JSON| -| --- | --- | - -## 六、脚本示例 - 见项目内`**chunjun-examples**`文件夹。 diff --git a/docs/connectors/tidb/tidb-sink.md b/docs/connectors/tidb/tidb-sink.md deleted file mode 100644 index 705937f17e..0000000000 --- a/docs/connectors/tidb/tidb-sink.md +++ /dev/null @@ -1,258 +0,0 @@ -# TiDB Sink - -## 一、介绍 -TiDB Sink插件支持向TiDB数据库写入数据 - -## 二、支持版本 -TiDB 3.0.10之后、TiDB 4.0 - -## 三、插件名称 -| Sync | tidbsink、tidbwriter | -| --- | --- | -| SQL | tidb-x | - -## 四、插件参数 - - -### 1.Sync - -- **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - - 必选:是 - - 字段类型:List - - 示例:指定jdbcUrl、schema、table -```json -"connection": [{ - "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", - "table": ["table"], - "schema":"public" - }] -``` - - - 默认值:无 - -​
- -- **jdbcUrl** - - 描述:TiDB兼容MySQL JDBC驱动 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 字段类型:String - - 默认值:无 - - -
- -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 字段类型:List - - 默认值:无 - - -
- -- **username** - - 描述:数据源的用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **password** - - 描述:数据源指定用户名的密码 - - 必选:是 - - 字段类型:String - - 默认值:无 - - -
- -- **column** - - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - - 必选:是 - - 字段类型:List - - 默认值:无 - - -
- -- **fullcolumn** - - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 - - 必选:否 - - 字段类型:List - - 默认值:无 - - -
- -- **preSql** - - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - - 必选:否 - - 字段类型:List - - 默认值:无 - - -
- -- **postSql** - - 描述:写入数据到目的表后,会执行这里的一组标准语句 - - 必选:否 - - 字段类型:List - - 默认值:无 - - -
- -- **writeMode** - - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - - 必选:是 - - 所有选项:insert/replace/update - - 字段类型:String - - 默认值:insert - - -
- -- **batchSize** - - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 - - 必选:否 - - 字段类型:int - - 默认值:1024 - - -
- -- **semantic** - - 描述:sink端是否支持二阶段提交 - - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once - - 必选:否 - - 参数类型:String - - 示例:"semantic": "exactly-once" - - 默认值:at-least-once -
- -- **updateKey** - - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 - - 注意: - - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - - 必选:否 - - 字段类型:Map - - 示例:"updateKey": {"key": ["id"]} - - 默认值:无 - - - -### 2.SQL - -- **connector** - - 描述:connector type - - 必选:是 - - 字段类型:String - - 值:sqlserver-x - -​
- -- **url** - - 描述:TiDB兼容MySQL JDBC连接驱动 - - 必选:是 - - 字段类型:String - - 默认值:无 - -​
- -- **table-name** - - 描述:表名 - - 必选:是 - - 字段类型:String - - 默认值:无 - -​
- -- **schema** - - 描述:数据库schema名 - - 必选:否 - - 字段类型:String - - 默认值:无 - -​
- -- **username** - - 描述:用户名 - - 必选:是 - - 字段类型:String - - 默认值:无 - -​
- -- **password** - - 描述:密码 - - 必选:是 - - 字段类型:String - - 默认值:无 - -​
- -- **sink.buffer-flush.max-rows** - - 描述:批量写数据条数,单位:条 - - 必选:否 - - 参数类型:String - - 默认值:1024 - -​
- -- **sink.buffer-flush.interval** - - 描述:批量写时间间隔,单位:毫秒 - - 必选:否 - - 参数类型:String - - 默认值:10000 - -
- -- **sink.all-replace** - - 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null) - - 必选:否 - - 参数类型:String - - 默认值:false - -
- -- **sink.semantic** - - 描述:sink端是否支持二阶段提交 - - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once - - 必选:否 - - 参数类型:String - - 示例:"semantic": "exactly-once" - - 默认值:at-least-once -
- -- **sink.parallelism** - - 描述:写入结果的并行度 - - 必选:否 - - 参数类型:String - - 默认值:无 - - - -## 五、数据类型 -|支持 | BIT、BOOL、BOOLEAN、SMALLINT、MEDIUMINT、INT、INTEGER、BIGINT、FLOAT、DOUBLE、DECIMAL、DATE、TIME、DATETIME、TIMESTAMP、YEAR、CHAR、VARCHAR、TEXT、TINYTEXT、MEDIUMTEXT、LONGTEXT、BINARY、VARBINARY、BLOB、TINYBLOB、MEDIUMBLOB、LONGBLOB、ENUM、SET、JSON| -| --- | --- | - - -## 六、脚本示例 - 见项目内`**chunjun-examples**`文件夹。 diff --git a/docs/contribution.md b/docs/contribution.md deleted file mode 100644 index 948ffcec87..0000000000 --- a/docs/contribution.md +++ /dev/null @@ -1,514 +0,0 @@ -# 如何贡献ChunJun - -本文面向ChunJun插件开发人员,尝试通过一个开发者的角度尽可能全面地阐述一个ChunJun插件所经历的过程,同时消除开发者的困惑,快速上手插件开发。 - -从数据流的角度来看ChunJun,可以理解为不同数据源的数据流通过对应的ChunJun插件处理,变成符合ChunJun数据规范的数据流;脏数据的处理可以理解为脏水流通过污水处理厂,变成符合标准,可以使用的水流,而对不能处理的水流收集起来。 - -插件开发不需要关注任务具体如何调度,只需要关注关键问题: - -1. 数据源本身读写数据的正确性; -1. 如何合理且正确地使用框架; -1. 配置文件的规范; - - -## 开发环境 - -- Flink集群: 1.4及以上(单机模式不需要安装Flink集群) -- Java: JDK8及以上 -- 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 - -开发之前,需要理解以下概念: - - -## 逻辑执行概念 -插件开发者不需要关心太多整个框架的具体运行,只需要关注数据源的读写,以及代码在逻辑上是怎么被执行的,方法什么时候被调用的。以下概念的理解对你快速开发会有帮助: - -- **Job**:** Job**是ChunJun用以描述从一个源头到一个目的端的同步作业,是ChunJun数据同步的最小业务单元。 -- **Internal**: 把**Job**拆分得到的最小执行单元。 -- **InputSplit**:数据切片,是进入Internal的最小数据流单位。里面包含了基本数据信息和统计信息。 -- **InputFormat**:读插件的执行单位。 -- **OutputFormat**:写插件的执行单位。 - - -## 任务执行模式 - -- 单机模式:对应Flink集群的单机模式 -- standalone模式:对应Flink集群的分布式模式 -- yarn模式:对应Flink集群的yarn模式 -- yarnPer模式: 对应Flink集群的Per-job模式 - -在实际开发中,上述几种模式对插件的编写没有过多的影响,一般在本地LocalTest通过,将任务上传到Flink集群测试没有什么大问题。 - - -## 插件入口类 -插件的入口类需继承**DataReader**和**DataWriter**,在内部获取任务json传来的参数,通过相应的**Builder**构建对应**InputFormat**和**OutputFormat**实例 - - -### DataReader - -```java -public class SomeReader extends DataReader { - protected String oneParameter; - public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - } - @Override - public DataStream readData() { - return null; - } -} -``` - -reader类需继承DataReader,同时重写readData方法。在构造函数中获取任务json中构建InputFormat所需要的参数,代码案例如下: - -构造方法 - -```java -protected String oneParameter; -public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - // 首先通过jobconfig获取任务json中reader信息 - ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - // 通过getParameter()获取相应的参数信息 - oneParameter = readerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER); -} -``` - -重写readData方法 - -```java -@Override -public DataStream readData() { - // 通过Builder构建InputFormat - SomeInputFormatBuilder builder = new SomeInputFormatBuilder(new SomeInputFormat()); - // 一个setOneParameter()方法只set一个参数 - builder.setOneParameter(OneParameter); - //调用createInput返回一个DataStream实例 - return createInput(builder.finish()); -} -``` - - -### DataWriter - -```java -public class SomeWriter extends DataWriter { - protected String oneParameter; - public SomeWriter(DataTransferConfig config) { - super(config); - } - - @Override - public DataStreamSink writeData(DataStream dataSet) { - return null; - } -} -``` - -和DataReader类似,writer需继承DataWriter,同时重写writeData方法。通常会创建一个ConfigKeys类,包含reader和writer所有需要的使用的任务json中参数的key。 - -构造方法 - -```java -protected String oneParameter; -public SomeWriter(DataTransferConfig config) { - super(config); - // 首先通过jobconfig获取jobjson中writer信息 - WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); - oneParameter = writerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER); -} -``` - -重写writeData方法 - -```java -@Override -public DataStreamSink writeData(DataStream dataSet) { - // 通过Builder构建OutputFormat - SomeOutputFormatBuilder builder = new SomeOutputFormatBuilder(new SomeOutputFormat()); - // 一个setOneParameter()方法只set一个参数 - builder.setOneParameter(OneParameter); - //调用createInput返回一个DataSink实例 - return createInput(builder.finish()); -} -``` - - -### InputFormatBuilder的设计 - -需继承**RichInputFormatBuilder** - -```java -public class SomeInputFormatBuilder extends RichInputFormatBuilder { - /** - * 首先实例化一个InputFormat实例,通过构造函数传递,通过set方法设置参数 - */ - protected SomeInputFormat format; - //InputFormat构造函数,需要给实例化父类的format - public SomeInputFormatBuilder(SomeInputFormat format){ - super.format = this.format = format; - } - //set方法示例,建议set方法内只给一个变量赋值 - public void setOneParameter(String oneParameter){ - this.oneParameter = oneParameter; - } - //重写checkFormat,检查一些必要参数设置是否正确 - @Override - protected void checkFormat() {} -} -``` - - -### InputFormat的设计 - -需继承**RichInputFormat**,根据任务逻辑分别实现 - -```java -public class SomeInputFormat extends RichInputFormat { - @override - public void openInputFormat() { - - } - - @override - public void closeInputFormat() { - } - - @Override - protected void openInternal(InputSplit inputSplit) throws IOException { - } - - @Override - protected Row nextRecordInternal(Row row) throws IOException { - return null; - } - - @Override - protected void closeInternal() throws IOException { - } - - @Override - public void configure(Configuration parameters) { - } - - @Override - public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - return new InputSplit[0]; - } - - @Override - public boolean reachedEnd() throws IOException { - return false; - } -} -``` - - -方法功能如下: - -#### configure - -- 调用位置:configure方法会在JobManager里构建执行计划的时候和在TaskManager里初始化并发实例后各调用一次; -- 作用:用于配置task的实例; -- 注意事项:不要在这个方法里写耗时的逻辑,比如获取连接,运行sql等,否则可能会导致akka超 - -#### createInputSplits - -- 调用位置:在构建执行计划时调用; -- 作用:调用子类的逻辑生成数据分片; -- 注意事项:分片的数量和并发数没有严格对应关系,不要在这个方法里做耗时的操作,否则会导致akka超时异常; - -#### getInputSplitAssigner - -- 调用位置:创建分片后调用; -- 作用:获取分片分配器,同步插件里使用的是DefaultInputSplitAssigner,按顺序返回分配给各个并发实例; -- 注意事项:无; - -#### openInternal - -- 调用位置:开始读取分片时调用; -- 作用:用于打开需要读取的数据源,并做一些初始化; -- 注意事项:这个方法必须是可以重复调用的,因为同一个并发实例可能会处理多个分片; - -#### reachEnd和nextRecordInternal - -- 调用位置:任务运行时,读取每条数据时调用; -- 作用:返回结束标识和下一条记录; -- 注意事项:无 - -#### closeInternal - -- 调用位置:读取完一个分片后调用,至少调用一次; -- 作用:关闭资源; -- 注意事项:可重复调用,关闭资源做非null检查,因为程序遇到异常情况可能直接跳转到closeInternal; - -#### openInputFormat - -- 调用位置:创建分片之后调用; -- 作用:对整个InpurFormat资源做初始化; -- 注意事项:无; - -#### closeInputFormat - -- 调用位置:当所有切片都执行完之后调用; -- 作用:关闭整个InputFormat的资源; -- 注意事项:无; - - -### OutputFormatBuilder -需继承**RichOutputFormatBuilder**,和**InputFormatBuilder**相似 - -```java -public class SomeOutputFormatBuilder extends RichOutputFormatBuilder { - /** - * 首先实例化一个OutputFormat实例,通过构造函数传递,通过设计set方法设置参数 - * 如下演示 - */ - protected SomeOutputFormat format; - - public SomeOutputFormatBuilder(SomeOutputFormat format){ - super.format = this.format = format; - } - - public void setOneParameter(String oneParameter){ - this.oneParameter = oneParameter; - } - - //重写checkFormat,检查参数设置是否正确 - @Override - protected void checkFormat() {} -} -``` - - -### OutputFormat -需继承**RichOutputFormat** - -```java -public class SomeOutputFormat extends RichOutputFormat { - @Override - protected void openInternal(int taskNumber, int numTasks) throws IOException {} - - @Override - protected void writeSingleRecordInternal(Record record) { - } - - @Override - protected void writeMultipleRecordsInternal() throws Exception { - } -} -``` - -各方法的执行逻辑如下: - -openInternal -> writeSingleRecordInternal / writeMultipleRecordsInternal - -对于是执行writeSingleRecordInternal 还是writeMultipleRecordsInternal,关键参数是batchInterval,当batchInterval=1 时,框架调用writeSingleRecordInternal;当batchInterval > 1 且 record != null时,则调用writeMultipleRecordsInternal - -方法功能如下: - -#### openInternal - -- 调用位置:开始写入使用 -- 作用:用于打开需要读取的数据源,并做一些初始化; -- 注意事项:无; - -#### writerSingleRecordInternal - -- 调用位置:openInernal之后调用,开始写入数据 -- 作用:向数据源写入一条数据 -- 注意事项:无; - -#### writerMultipleRecordsInternal - -- 调用位置:openInternal之后调用,开始写入多条数据 -- 作用:向数据源写入多条数据,由batchInterval参数决定写入多少条 -- 注意事项:无; - - -## ChunJun数据结构 -ChunJun延续了Flink原生的数据类型Row - -```java -@PublicEvolving -public class Row implements Serializable{ - - private static final long serialVersionUID = 1L; - - /** The array to store actual values. */ - private final Object[] fields; - - /** - * Create a new Row instance. - * @param arity The number of fields in the Row - */ - public Row(int arity) { - this.fields = new Object[arity]; - } -} -``` - - -## 任务json配置 -配置中尽量减少不必要的参数,有些参数框架中已有默认值,配置文件中的值优先,模板如下 - -```json -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "username": "", - "password": "", - "connection": [{ - "jdbcUrl": [""], - "table": [ - "" - ] - }], - "column": [{ - "name": "id", - "type": "int" - },{ - "name":"name", - "type":"string" - }] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print": true - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - }, - "speed": { - } - } - } -} -``` - - -## 如何设计配置参数 -任务配置中`reader`和`writer`下`parameter`部分是插件的配置参数,插件的配置参数应当遵循以下原则: - -- 驼峰命名:所有配置项采用驼峰命名法,首字母小写,单词首字母大写。 -- 正交原则:配置项必须正交,功能没有重复,没有潜规则。 -- 富类型:合理使用json的类型,减少无谓的处理逻辑,减少出错的可能。 - - 使用正确的数据类型。比如,bool类型的值使用`true`/`false`,而非`"yes"`/`"true"`/`0`等。 - - 合理使用集合类型,比如,用数组替代有分隔符的字符串。 -- 类似通用:遵守同一类型的插件的习惯,比如关系型数据库的`connection`参数都是如下结构: -``` -{ - "connection": [ - { - "table": [ - "table_1", - "table_2" - ], - "jdbcUrl": [ - "jdbc:mysql://127.0.0.1:3306/database_1", - "jdbc:mysql://127.0.0.2:3306/database_1_slave" - ] - }, - { - "table": [ - "table_3", - "table_4" - ], - "jdbcUrl": [ - "jdbc:mysql://127.0.0.3:3306/database_2", - "jdbc:mysql://127.0.0.4:3306/database_2_slave" - ] - } - ] -} -``` - - -## 如何处理脏数据 - -### 脏数据定义 - -1. Reader读到不支持的类型、不合法的值。 -1. 不支持的类型转换,比如:`Bytes`转换为`Date`。 -1. 写入目标端失败,比如:写mysql整型长度超长。 - - -### 如何处理脏数据 -框架会将脏数据临时存放起来。由DirtyDataManager实例写入临时存放脏数据文件中。 - -- path: 脏数据存放路径 -- hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置) - - -## 加载原理 - -1. 框架扫描`plugin/reader`和`plugin/writer`目录,加载每个插件的`plugin.json`文件。 -1. 以`plugin.json`文件中`name`为key,索引所有的插件配置。如果发现重名的插件或者不存在的插件,框架会异常退出。 -1. 用户在插件中在`reader`/`writer`配置的`name`字段指定插件名字。框架根据插件的类型(`reader`/`writer`)和插件名称去插件的路径下扫描所有的jar,加入`classpath`。 -1. 根据插件配置中定义的入口类,框架通过反射实例化对应的`Job`对象。 - - -## 统一的目录结构 - -#### 项目目录层级 -注意,插件Reader/Writer类需放在符合插件包名命名规则的reader下,如MysqlReader类需放在com.dtstack.chunjun.mysql.reader包下,具体命名规则参照 **项目命名规则** 内容 -```xml -``` -${CHUNJUN_HOME} -|-- bin -| -- flink -| -- chunjun.sh -| -|-- chunjun-somePlugin - |-- chunjun-somePlugin-core - |-- common 一些插件共用的类 - |-- exception 异常处理类 - |-- pom.xml 插件公用依赖 - |-- chunjun-somePlugin-reader - |-- InputFormat - |-- SomePluginInputFormat - |-- SomePluginInputFormatBuiler - |-- reader - |-- SomePluginReader - |-- chunjun-somePlugin-writer - |-- OutputFormat - |-- SomePluginOutputFormat - |-- SomePluginOutputFormatBuiler - |-- reader - |-- SomePluginWriter -``` -``` - -#### 项目命名规则 - -- 插件命名模板 [chunjun]-[dataSourceName],例如chunjun-mysql -- 插件模块命名模板 [chunjun]-[dataSourceName]-[reader/writer/core],例如chunjun-mysql-reader,chunjun-redis-writer -- 插件包名命名模板 [com.dtstack.chunjun.dataSource.xxxx],例如com.dtstack.chunjun.mysql.reader,com.dtstack.chunjun.redis.inputformat -- 插件Reader/Writer类命名模板 [dataSource][Reader/Writer],例如MysqlReader,RedisWriter,需注意,类似RestAPIWriter,MetaDataHive2Reader这样的命名是错误的,需改为RestapiWriter,Metadatahive2Reader - - -## 插件打包 -进入项目根目录,使用maven打包: - -windows平台 - -``` -mvn clean package -DskipTests -Prelease -DscriptType=bat -``` - -unix平台 - -``` -mvn clean package -DskipTests -Prelease -DscriptType=sh -``` - -打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含ChunJun的启动脚本,plugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! diff --git a/docs/generalconfig.md b/docs/generalconfig.md deleted file mode 100644 index 5d5f0df206..0000000000 --- a/docs/generalconfig.md +++ /dev/null @@ -1,193 +0,0 @@ -# 插件通用配置 - -## 一、Sync - -### 配置文件 -一个完整的ChunJun任务脚本配置包含 content,setting两个部分。content用于配置任务的输入源、输出源以及数据转换规则,其中包含reader,writer,transformer。setting则配置任务整体的环境设定,其中包含speed,errorLimit,dirty。具体如下所示: -```json -{ - "job": { - "content": [ - { - "reader": {}, - "writer": {}, - "transformer": {} - } - ], - "setting": { - "speed": {}, - "errorLimit": {}, - "dirty": {} - } - } -} -``` - - -| 名称 | | 说明 | 是否必填 | -| --- | --- | --- | --- | -| content | reader | reader插件详细配置 | 是 | -| | writer | writer插件详细配置 | 是 | -| | transformer | 数据转换SQL | 否 | -| setting | speed | 速率限制 | 否 | -| | errorLimit | 出错控制 | 否 | -| | dirty | 脏数据保存 | 否 | - - -### content配置 - -#### reader -reader用于配置数据的输入源,即数据从何而来。具体配置如下所示: - -```json -{ -"reader" : { - "name" : "xxreader", - "parameter" : {}, - "table": { - "tableName": "xxx" - } - } -} -``` - -| 名称 | 说明 | 是否必填 | -| --- | --- | --- | -| name | reader插件名称,具体名称参考各数据源配置文档 | 是 | -| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | -| table | SQL源表名称 | 开启transformer后必填 | - - - -### writer -writer用于配置数据的输出源,即数据写往何处。具体配置如下所示: - -```json -{ -"writer" : { - "name" : "xxwriter", - "parameter" : {}, - "table": { - "tableName": "xxx" - } - } -} -``` -| 名称 | 说明 | 是否必填 | -| --- | --- | --- | -| name | writer插件名称,具体名称参考各数据源配置文档 | 是 | -| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | -| table | SQL结果表名称 | 开启transformer后必填 | - -### transformer配置 -transformer用于配置数据转换SQL,支持所有Flink原生语法及Function - -```json -{ -"transformer" : { - "transformSql": "xxx" - } -} -``` - - -### setting配置 - -#### speed -speed用于配置任务并发数及速率限制。具体配置如下所示: - -```json -{ -"speed" : { - "channel": 1, - "readerChannel": 2, - "writerChannel": 2, - "bytes": 0 -} -} -``` -| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | -| --- | --- | --- | --- | --- | -| channel | 整体任务并行度 | 否 | 1 | int | -| readerChannel | source并行度 | 否 | -1 | int | -| writerChannel | sink并行度 | 否 | -1 | int | -| bytes | bytes >0则表示开启任务限速 | 否 | 0 | Long | - - -#### errorLimit -errorLimit用于配置任务运行时数据读取写入的出错控制。具体配置如下所示: - -```json -{ -"errorLimit" : { - "record": 100, - "percentage": 10.0 -} -} -``` -| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | -| --- | --- | --- | --- | --- | -| record | 错误阈值,当错误记录数超过此阈值时任务失败 | 否 | 0 | int | -| percentage | 错误比例阈值,当错误记录比例超过此阈值时任务失败 | 否 | 0.0 | Double | - -#### -metricPluginConf用于配置任务运行时自定义指标持久化的方式。具体配置如下所示: - -```json -{ -"metricPluginConf":{ - "pluginName": "mysql", - "pluginProp": { - "jdbcUrl": "jdbc:mysql://localhost:3306/ide?useUnicode=true&characterEncoding=utf-8", - "schema": "ide", - "table": "chunjun_metrics", - "username": "drpeco", - "password": "DT@Stack#123" - } -} -} -``` -| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | -| --- | --- | --- | --- | --- | -| pluginName | 持久化插件的名称 | 否 | prometheus | String | -| pluginProp | 连接插件需要用到的参数配置 | 否 | 无 | Map | - - -#### dirty -dirty用于配置脏数据的保存,通常与上文出错控制联合使用。具体配置如下所示: - -```json -{ -"dirty" : { - "path" : "xxx", - "hadoopConfig" : { - } - } -} -``` -| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | -| --- | --- | --- | --- | --- | -| path | 脏数据保存路径 | 是 | 无 | Sring | -| hadoopConfig | Hadoop相关配置 | 是 | 无 | K-V键值对 | - - -参考模板如下: - -```json -{ -"dirty" : { - "path" : "/user/hive/warehouse/xx.db/xx", - "hadoopConfig" : { - "fs.default.name": "hdfs://0.0.0.0:9000", - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1" : "0.0.0.0:9000", - "dfs.namenode.rpc-address.ns1.nn2" : "0.0.0.1:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.nameservices" : "ns1" - } - } -} -``` -## 二、SQL -[参考Flink官方文档](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/sql/) - diff --git a/docs/iceberg.md b/docs/iceberg.md deleted file mode 100644 index 7e6b4ab0af..0000000000 --- a/docs/iceberg.md +++ /dev/null @@ -1,5 +0,0 @@ -基于Iceberg master分支打包,可以在Flink 1.12下流式读写Iceberg。 - -[SQL 样例](chunjun-examples/sql/iceberg/iceberg_streaming.sql) - -后续支持使用JSON配置ChunJun数据同步任务,将数据入湖出湖。 diff --git a/docs/images/IMG_3362.JPG b/docs/images/IMG_3362.JPG deleted file mode 100644 index 45377cf1fc..0000000000 Binary files a/docs/images/IMG_3362.JPG and /dev/null differ diff --git a/docs/images/SqlserverCDC/Sqlserver15.png b/docs/images/SqlserverCDC/Sqlserver15.png deleted file mode 100644 index 3345aaadda..0000000000 Binary files a/docs/images/SqlserverCDC/Sqlserver15.png and /dev/null differ diff --git a/docs/images/StarRocks/StarRocks1.png b/docs/images/StarRocks/StarRocks1.png deleted file mode 100644 index 6c021587fa..0000000000 Binary files a/docs/images/StarRocks/StarRocks1.png and /dev/null differ diff --git a/docs/images/conectorShare_local.png b/docs/images/conectorShare_local.png deleted file mode 100644 index cffd860ff5..0000000000 Binary files a/docs/images/conectorShare_local.png and /dev/null differ diff --git a/docs/images/conectorShare_yarn.png b/docs/images/conectorShare_yarn.png deleted file mode 100644 index 6d2b87678a..0000000000 Binary files a/docs/images/conectorShare_yarn.png and /dev/null differ diff --git a/docs/images/conectorShare_yarnPer.png b/docs/images/conectorShare_yarnPer.png deleted file mode 100644 index 1e11bd6d37..0000000000 Binary files a/docs/images/conectorShare_yarnPer.png and /dev/null differ diff --git a/docs/images/diagram.png b/docs/images/diagram.png deleted file mode 100644 index 17a94dc7dc..0000000000 Binary files a/docs/images/diagram.png and /dev/null differ diff --git a/docs/images/restore1.png b/docs/images/restore1.png deleted file mode 100644 index 7104d7f2b1..0000000000 Binary files a/docs/images/restore1.png and /dev/null differ diff --git a/docs/images/restore2.png b/docs/images/restore2.png deleted file mode 100644 index fdcb071904..0000000000 Binary files a/docs/images/restore2.png and /dev/null differ diff --git a/docs/images/restore3.jpg b/docs/images/restore3.jpg deleted file mode 100644 index c1810b092e..0000000000 Binary files a/docs/images/restore3.jpg and /dev/null differ diff --git a/docs/images/restore4.png b/docs/images/restore4.png deleted file mode 100644 index 39aa8ca0cf..0000000000 Binary files a/docs/images/restore4.png and /dev/null differ diff --git a/docs/images/restore5.png b/docs/images/restore5.png deleted file mode 100644 index 597bef156e..0000000000 Binary files a/docs/images/restore5.png and /dev/null differ diff --git a/docs/images/template.png b/docs/images/template.png deleted file mode 100644 index 285f893327..0000000000 Binary files a/docs/images/template.png and /dev/null differ diff --git a/docs/kerberos.md b/docs/kerberos.md deleted file mode 100644 index 1a0dd5dff3..0000000000 --- a/docs/kerberos.md +++ /dev/null @@ -1,115 +0,0 @@ -# 数据源开启Kerberos安全认证 - -目前ChunJun的部分插件支持了kerberos认证,有Hive、Hbase、HDFS三个插件。 - -### 1.Kerberos证书加载方式 - -目前支持两种方式,一种是从本地加载,即任务运行的机器上对应的目录必须存在配置里指定的证书文件,另一种是从sftp服务器下载,需要配置sftp服务器的配置信息。 - -使用本地配置示例: - -```json -"hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "host1,host2,host3", - "zookeeper.znode.parent": "/hbase", - "hbase.security.authentication":"Kerberos", - "hbase.security.authorization":true, - "hbase.master.kerberos.principal":"hbase/node1@TEST.COM", - "hbase.master.keytab.file":"hbase.keytab", - "hbase.regionserver.keytab.file":"hbase.keytab", - "hbase.regionserver.kerberos.principal":"hbase/node1@TEST.COM", - "java.security.krb5.conf":"krb5.conf", - "useLocalFile":true - } -``` - -从sftp下载配置示例: - -```json -"hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "host1,host2,host3", - "zookeeper.znode.parent": "/hbase", - "hbase.security.authentication":"Kerberos", - "hbase.security.authorization":true, - "hbase.master.kerberos.principal":"hbase/node1@TEST.COM", - "hbase.master.keytab.file":"hbase.keytab", - "hbase.regionserver.keytab.file":"hbase.keytab", - "hbase.regionserver.kerberos.principal":"hbase/node1@TEST.COM", - "remoteDir":"/sftp/chunjun/keytab/hbase", - "sftp":{ - "host":"127.0.0.1", - "port":"22", - "username":"", - "password":"" - } - } -``` - -从sftp下载时的查找顺序: - -1.在/sftp/chunjun/keytab/hbase目录下查找hbase.keytab文件,如果找不到则2 - -2.假设任务运行在node1机器上,则在/sftp/chunjun/keytab/hbase/node1下找hbase.keytab文件,找不到则报错; - -### 2.各数据源的配置 - -#### hbase - -```json -"hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "host1,host2,host3", - "zookeeper.znode.parent": "/hbase", - "hbase.security.authentication":"Kerberos", - "hbase.security.authorization":true, - "hbase.master.kerberos.principal":"hbase/node1@TEST.COM", - "hbase.master.keytab.file":"hbase.keytab", - "hbase.regionserver.keytab.file":"hbase.keytab", - "hbase.regionserver.kerberos.principal":"hbase/node1@TEST.COM", - "java.security.krb5.conf":"krb5.conf" - } -``` - -#### hive - -```json -"hadoopConf":{ - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" - "hadoop.security.authorization": "true", - "hadoop.security.authentication": "Kerberos", - "dfs.namenode.kerberos.principal": "hdfs/_HOST@HADOOP.COM", - "dfs.namenode.keytab.file": "hdfs.keytab", - "java.security.krb5.conf": "krb5.conf" -} -``` - -jdbcUrl格式:jdbc:hive2://127.0.0.1:10000/default;principal=hive/node1@HADOOP.COM - -#### hdfs - -```json -"hadoopConf":{ - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" - "hadoop.security.authorization": "true", - "hadoop.security.authentication": "Kerberos", - "dfs.namenode.kerberos.principal": "hdfs/_HOST@HADOOP.COM", - "dfs.namenode.keytab.file": "hdfs.keytab", - "java.security.krb5.conf": "krb5.conf" -} -``` diff --git a/docs/prometheus.md b/docs/prometheus.md deleted file mode 100644 index 6756e7d365..0000000000 --- a/docs/prometheus.md +++ /dev/null @@ -1,7 +0,0 @@ -## 使用 prometheus pushgateway 需要设置的 confProp 参数 -* metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter -* metrics.reporter.promgateway.host: prometheus pushgateway的地址 -* metrics.reporter.promgateway.port:prometheus pushgateway的端口 -* metrics.reporter.promgateway.jobName: 实例名称 -* metrics.reporter.promgateway.randomJobNameSuffix: 是否在实例名称后面添加随机字符串(默认:true) -* metrics.reporter.promgateway.deleteOnShutdown: 是否在停止的时候删除数据(默认false) diff --git a/docs/questions.md b/docs/questions.md deleted file mode 100644 index 17a00475d2..0000000000 --- a/docs/questions.md +++ /dev/null @@ -1,45 +0,0 @@ -## 常见问题 - -### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 - -解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: - -```bash -## windows平台 -./install_jars.bat - -## unix平台 -./install_jars.sh -``` - -### 2.ChunJun版本需要与Flink版本保持一致 -1.8_release版本对应flink1.8 -1.10_release版本对应flink1.10 版本 -不对应在standalone和yarn session模式提交时,会报错: -Caused by: java.io.InvalidClassException: org.apache.flink.api.common.operators.ResourceSpec; incompatible types for field cpuCores - -### 3.移动ChunJun lib目录下的Launcher包后,任务启动报错:错误: 找不到或无法加载主类 -chunjun启动脚本里面找的是lib目录下的所有jar包,而移动后lib中含有其他的jar包,这些jar包没有主类,因此报错 -可以使用如下的命令运行: -java -cp chunjun/lib/chunjun-client.jar com.dtstack.chunjun.launcher.Launcher -mode local -job /opt/flink/flink/deps/job/stream.json -chunjunDistDir chunjun/chunjun-dist - -### 4. 关于编译chunjun core报错Failed to read artifact descriptor for com.google.errorprone:javac-shaded - -报错信息: - -``` -[ERROR] Failed to execute goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check (spotless-check) on project chunjun-core: -Execution spotless-check of goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check failed: Unable to resolve dependencies: -Failed to collect dependencies at com.google.googlejavaformat:google-java-format:jar:1.7 -> com.google.errorprone:javac-shaded:jar:9+181-r4173-1: -Failed to read artifact descriptor for com.google.errorprone:javac-shaded:jar:9+181-r4173-1: Could not transfer artifact -com.google.errorprone:javac-shaded:pom:9+181-r4173-1 from/to aliyunmaven (https://maven.aliyun.com/repository/public): -Access denied to:https://maven.aliyun.com/repository/public/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.pom -> [Help 1] -``` - -解决: - -https://repo1.maven.org/maven2/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.jar -从这个地址下载javac-shaded-9+181-r4173-1.jar, 临时放到chunjun根目录下jars目录里,然后在源码根目录下 执行安装依赖包命令如下: - -``` -mvn install:install-file -DgroupId=com.google.errorprone -DartifactId=javac-shaded -Dversion=9+181-r4173-1 -Dpackaging=jar -Dfile=./jars/javac-shaded-9+181-r4173-1.jar diff --git a/docs/quickstart.md b/docs/quickstart.md deleted file mode 100644 index 4a038607ec..0000000000 --- a/docs/quickstart.md +++ /dev/null @@ -1,448 +0,0 @@ -## 下载代码 - -1.使用git工具把项目clone到本地 - -``` -git clone https://github.com/DTStack/chunjun.git -cd chunjun -``` - -## 编译插件 -在chunjun home目录下执行 - -```bash -mvn clean package -DskipTests -``` -或者执行 -```bash -sh build/build.sh -``` - -## 常见问题 - -### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 - -解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: - -```bash -## windows平台 -./$CHUNJUN_HOME/bin/install_jars.bat - -## unix平台 -./$CHUNJUN_HOME/bin/install_jars.sh -``` -## 安装包快速部署 -执行完上述命令之后,在chunjun-assembly模块的target目录会得到一个完整的安装包,快速部署只需要将这个安装包解压即可使用。 -
- -
-将安装包上传至安装服务器解压即可,执行如下命令: - -```bash -# 创建一个目录 -mkdir /opt/chunjun -# 解压 -tar -zxvf chunjun-assembly-XXX-chunjun-dist.tar.gz -C /opt/chunjun - -cd /opt/chunjun - -``` -解压之后,进入chunjun目录能够看到如下目录和文件 -
- -
- -```bash -# 配置CHUNJUN_HOME,在/etc/profile 最后一行添加如下内容 - -export CHUNUN_HOME=/opt/chunjun - -# 添加好之后,保存并使其生效 -source /etc/profile - -``` - - -快速执行入门案例,验证安装结果 - -```bash -# 创建一个目录 -bin/start-chunjun \ - -mode local \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME - -# 查看结果 -tail -f nohup.out -``` -效果如下: -
- -
- -## 常见问题 -### 执行命令出现权限不足 -执行start-chunjun命令时,出现-bash: bin/start-chunjun: Permission denied,如图所示 -
- -
-这是由于bin目录下的脚本文件没有可执行权限,赋予可执行权限即可 - -```bash -cd bin -# 给bin目录下的所有文件赋予可执行文件 -chmod u+x * -``` - - -## 运行任务 -**NOTE:项目中的chunjun-examples模块下提供了大量 [数据同步案例](chunjun-examples/json) 和 [SQL案例](chunjun-examples/sql)** - -#### 数据同步任务 - -首先准备要运行的任务json,这里以stream插件为例(**`chunjun-examples`文件夹下有大量案例**): - -```json -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "id" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "content", - "type": "string" - } - ], - "sliceRecordCount": ["30"], - "permitsPerSecond": 1 - }, - "table": { - "tableName": "sourceTable" - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "column": [ - { - "name": "id", - "type": "id" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "content", - "type": "string" - } - ], - "print": true - }, - "table": { - "tableName": "sinkTable" - }, - "name": "streamwriter" - }, - "transformer": { - "transformSql": "select id,name, content from sourceTable where CHAR_LENGTH(name) < 50 and CHAR_LENGTH(content) < 50" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 0, - "channel": 1, - "readerChannel": 1, - "writerChannel": 1 - } - } - } -} -``` -#### flinksql任务 - -***NOTE:ChunJun和flinkSql connector[共用](docs/conectorShare.md)***

-或者准备要运行的flinksql任务,这里以stream插件为例(**`chunjun-examples`文件夹下有大量案例**): - -```sql -CREATE TABLE source -( - id INT, - name STRING, - money DECIMAL(32, 2), - dateone timestamp, - age bigint, - datethree timestamp, - datesix timestamp(6), - datenigth timestamp(9), - dtdate date, - dttime time -) WITH ( - 'connector' = 'stream-x', - 'number-of-rows' = '10', -- 输入条数,默认无限 - 'rows-per-second' = '1' -- 每秒输入条数,默认不限制 - ); - -CREATE TABLE sink -( - id INT, - name STRING, - money DECIMAL(32, 2), - dateone timestamp, - age bigint, - datethree timestamp, - datesix timestamp(6), - datenigth timestamp(9), - dtdate date, - dttime time -) WITH ( - 'connector' = 'stream-x', - 'print' = 'true' - ); - -insert into sink -select * -from source; -``` - -### Local模式运行任务 - -命令模板: - -```bash -bin/start-chunjun \ - -mode local \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME -``` - -可以在flink-conf.yaml配置文件里配置端口: - -```bash -## web服务端口,不指定的话会随机生成一个 -rest.bind-port: 8888 -``` - -使用下面的命令运行任务: - -```bash -bin/start-chunjun \ - -mode local \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME -``` - -任务运行后可以通过8888端口访问flink界面查看任务运行情况: - -
- -
- -### Standalone模式运行 -NOTE:将chunjun-dist目录拷贝到$FLINK_HOME/lib下,并修改$FLINK_HOME/conf/flink-conf.yml中的classloader为classloader.resolve-order: parent-first - -命令模板: - -```bash -bin/start-chunjun \ - -mode standalone \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME \ - -flinkConfDir $FLINK_HOME/conf \ - -confProp "{\"flink.checkpoint.interval\":60000}" -``` - -首先启动flink集群: - -```bash -# flink集群默认端口是8081 -$FLINK_HOME/bin/start-cluster.sh -``` - -通过8081端口检查集群是否启动成功 - -
- -
- -把任务提交到集群上运行: - -```bash -./bin/start-chunjun \ - -mode standalone \ - -jobType sync \ - -chunjunDistDir $CHUNJUN_HOME \ - -job chunjun-examples/json/stream/stream.json \ - -flinkConfDir $FLINK_HOME/conf -``` - -在集群上查看任务运行情况 - -
- -
- -### 以Yarn Session模式运行任务 -NOTE:可以先在现在chunjun-clients模块YarnSessionClientUtil类中启动一个session,然后修改$FLINK_HOME/conf/flink-conf.yml中的classloader为classloader.resolve-order: parent-first - -命令示例: - -```bash -bin/start-chunjun \ - -mode yarn-session \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME \ - -flinkConfDir $FLINK_HOME/conf \ - -hadoopConfDir $HADOOP_HOME/etc/hadoop \ - -confProp "{\"flink.checkpoint.interval\":60000}" -``` - -首先确保yarn集群是可用的,然后手动启动一个yarn session: - -```bash -$FLINK_HOME/bin/yarn-session.sh -n 1 -s 1 -jm 1024 -tm 1024 -``` - -
- -
- -
- -
- -把任务提交到这个yarn session上: - -```bash -bin/start-chunjun \ - -mode yarn-session \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -flinkConfDir $FLINK_HOME/conf \ - -chunjunDistDir $CHUNJUN_HOME \ - -hadoopConfDir $HADOOP_HOME/etc/hadoop -``` - -然后在flink界面查看任务运行情况: - -
- -
- -### 以Yarn Perjob模式运行任务 - -命令示例: - -```bash -bin/start-chunjun \ - -mode yarn-per-job \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME \ - -flinkConfDir $FLINK_HOME/conf \ - -hadoopConfDir $HADOOP_HOME/etc/hadoop \ - -flinkLibDir $FLINK_HOME/lib \ - -confProp "{\"flink.checkpoint.interval\":60000,\"yarn.application.queue\":\"default\"}" \ -``` - -首先确保yarn集群是可用的,启动一个Yarn Application运行任务: - -```bash -bin/start-chunjun \ - -mode yarn-per-job \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -chunjunDistDir $CHUNJUN_HOME \ - -hadoopConfDir $HADOOP_HOME/etc/hadoop \ - -flinkLibDir $FLINK_HOME/lib \ -``` - -然后在集群上查看任务运行情况 - -
- -
- -
- -
- - -### Kubernetes Session模式运行任务 - -命令示例: - -``` -bin/start-chunjun \ - -mode kubernetes-session \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -jobName kubernetes-job \ - -jobType sync \ - -chunjunDistDir $CHUNJUN_HOME \ - -flinkLibDir $FLINK_HOME/lib \ - -flinkConfDir $FLINK_HOME/conf \ - -confProp "{\"kubernetes.config.file\":\"${kubernetes_config_path}\",\"kubernetes.cluster-id\":\"${cluster_id}\",\"kubernetes.namespace\":\"${namespace}\"}" -``` - -需要提前手动在kubernetes上启动kubernetes session -``` -$FLINK_HOME/bin/kubernetes-session.sh -Dkubernetes.cluster-id=flink-session-test -Dclassloader.resolve-order=parent-first -Dkubernetes.container.image=${image_name} -``` -注意:需要提前构建chunjun镜像 -[chunjun镜像构建说明](chunjun-docker/docker/README.md) - -### Kubernetes Application模式运行任务 - -命令示例: -``` -bin/start-chunjun \ - -mode kubernetes-application \ - -jobType sync \ - -job chunjun-examples/json/stream/stream.json \ - -jobName kubernetes-job \ - -jobType sync \ - -chunjunDistDir $CHUNJUN_HOME \ - -remotePluginPath /opt/chunjun-dist \ - -pluginLoadMode classpath \ - -flinkLibDir $FLINK_HOME/lib \ - -flinkConfDir $FLINK_HOME/conf \ - -confProp "{\"kubernetes.config.file\":\"${kubernetes_config_path}\",\"kubernetes.container.image\":\"${image_name}\",\"kubernetes.namespace\":\"${namespace}\"}" -``` -注意:需要提前构建chunjun镜像 -[chunjun镜像构建说明](chunjun-docker/docker/README.md) - -## 参数说明 - -| 名称 | 说明 | 可选值 | 是否必填 | 默认值 | -| ------------------ | ------------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---- | ----------------------- | -| **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn-session**: yarn-session模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarn-per-job**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster"
5.**kubernetes-session**: kubernetes session模式提交任务,需要提前在kubernetes上启动flink session
6.**kubernetes-application**: kubernetes run application模式提交任务 | 否 | local | -| **jobType** | 任务类型 | 1.**sync**:数据同步任务
2.**sql**:flinksql任务 | 是 | 无 | -| **job** | 同步、flinksql任务描述文件的存放路径;该描述文件中使用json、sql存放任务信息 | 无 | 是 | 无 | -| **jobName** | 任务名称 | 无 | 否 | Flink Job | -| **chunjunDistDir** | 插件根目录地址,也就是打包后产生的chunjun-dist目录。 | 无 | 否 | $CHUNJUN_HOME/chunjun-dist | -| **flinkConfDir** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | -| **flinkLibDir** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | -| **hadoopConfDir** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | -| **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点chunjun-dist目录下部署插件包,但任务启动速度较快,session模式建议使用
2.**shipfile**:提交任务时上传chunjun-dist目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境,yarnPer模式建议使用 | 否 | shipfile | -| **confProp** | flink官方所有配置参数 | | 否 | 无 | -| **p** | 自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102| | 否 | 无 | diff --git a/docs/restore.md b/docs/restore.md deleted file mode 100644 index bdc9ee0018..0000000000 --- a/docs/restore.md +++ /dev/null @@ -1,230 +0,0 @@ -### 1.功能介绍 - -#### **1.1** **断点续传** - -断点续传是指数据同步任务在运行过程中因各种原因导致任务失败,不需要重头同步数据,只需要从上次失败的位置继续同步即可,类似于下载文件时因网络原因失败,不需要重新下载文件,只需要继续下载就行,可以大大节省时间和计算资源。 - -#### **2.1** **实时采集** - -根据数据源的数据是否实时变化可以把数据同步分为离线数据同步和实时数据同步,上面介绍的断点续传就是离线数据同步里的功能,实时采集其实就是实时数据同步,当数据源里的数据发生了增删改操作,同步任务监听到这些变化,将变化的数据实时同步到目标数据源。除了数据实时变化外,实时采集和离线数据同步的另一个区别是:实时采集任务是不会停止的,任务会一直监听数据源是否有变化。 - -### 2.Flink**中的checkpoint机制** - -断点续传和实时采集都依赖于flink的checkpoint机制,这里简单介绍一下。 Checkpoint是Flink实现容错机制最核心的功能,它能够根据配置周期性地基于Stream中各个Operator的状态来生成Snapshot,从而将这些状态数据定期持久化存储下来,当Flink程序一旦意外崩溃时,重新运行程序时可以有选择地从这些Snapshot进行恢复,从而修正因为故障带来的程序数据状态中断。 - -
- -
- -Checkpoint触发时,会向多个分布式的Stream Source中插入一个Barrier标记,这些Barrier会随着Stream中的数据记录一起流向下游的各个Operator。当一个Operator接收到一个Barrier时,它会暂停处理Steam中新接收到的数据记录。因为一个Operator可能存在多个输入的Stream,而每个Stream中都会存在对应的Barrier,该Operator要等到所有的输入Stream中的Barrier都到达。当所有Stream中的Barrier都已经到达该Operator,这时所有的Barrier在时间上看来是同一个时刻点(表示已经对齐),在等待所有Barrier到达的过程中,Operator的Buffer中可能已经缓存了一些比Barrier早到达Operator的数据记录(Outgoing Records),这时该Operator会将数据记录(Outgoing Records)发射(Emit)出去,作为下游Operator的输入,最后将Barrier对应Snapshot发射(Emit)出去作为此次Checkpoint的结果数据。 - -### **3.断点续传** - -#### **3.1** **前提条件** - -同步任务要支持断点续传,对数据源有一些强制性的要求: - -- 数据源(这里特指关系数据库)中必须包含一个升序的字段,比如主键或者日期类型的字段,同步过程中会使用checkpoint机制记录这个字段的值,任务恢复运行时使用这个字段构造查询条件过滤已经同步过的数据,如果这个字段的值不是升序的,那么任务恢复时过滤的数据就是错误的,最终导致数据的缺失或重复; - -- 数据源必须支持数据过滤,如果不支持的话,任务就无法从断点处恢复运行,会导致数据重复; - -- 目标数据源必须支持事务,比如关系数据库,文件类型的数据源也可以通过临时文件的方式支持; - -#### **3.2** **任务运行的详细过程** - -我们用一个具体的任务详细介绍一下整个过程,任务详情如下: - -| **数据源** | mysql表,假设表名data_test,表中包含主键字段id | -| -------------------- | ---------------------------------------------------------------------- | -| **目标数据源** | hdfs文件系统,假设写入路径为 /data_test | -| **并发数** | 2 | -| **checkpoint****配置** | 时间间隔为60s,checkpoint的StateBackend为FsStateBackend,路径为 /chunjun/checkpoint | -| **jobId** | 用来构造数据文件的名称,假设为 abc123 | - -##### **1)** **读取数据** - -读取数据时首先要构造数据分片,构造数据分片就是根据通道索引和checkpoint记录的位置构造查询sql,sql模板如下: - -```sql -select * from data_test -where id mod ${channel_num}=${channel_index} -and id > ${offset} -``` - -如果是第一次运行,或者上一次任务失败时还没有触发checkpoint,那么offset就不存在,根据offset和通道可以确定具体的查询sql: - -offset存在时 - -```sql -#第一个通道: -select * from data_test -where id mod 2=0 -and id > ${offset_0}; - -#第二个通道: -select * from data_test -where id mod 2=1 -and id > ${offset_1}; -``` - -offset不存在时 - -```sql -#第一个通道: -select * from data_test -where id mod 2=0; - -#第二个通道: -select * from data_test -where id mod 2=1; -``` - -数据分片构造好之后,每个通道就根据自己的数据分片去读数据了。 - -##### **2**)写数据 - -写数据前会先做几个操作: - -1. 检测 /data_test 目录是否存在,如果目录不存在,则创建这个目录,如果目录存在,进行2操作; - -2. 判断是不是以覆盖模式写数据,如果是,则删除 /data_test目录,然后再创建目录,如果不是,则进行3操作; - -3. 检测 /data_test/.data 目录是否存在,如果存在就先删除,再创建,确保没有其它任务因异常失败遗留的脏数据文件; - - 数据写入hdfs是单条写入的,不支持批量写入。数据会先写入/data_test/.data/目录下,数据文件的命名格式为: channelIndex.jobId.fileIndex,包含通道索引,jobId,文件索引三个部分。 - -##### **3**)checkpoint触发时 - -在ChunJun中“状态”表示的是标识字段id的值,我们假设checkpoint触发时两个通道的读取和写入情况如图中所示: - -
- -
- -checkpoint触发后,两个reader先生成Snapshot记录读取状态,通道0的状态为 id=12,通道1的状态为 id=11。Snapshot生成之后向数据流里面插入barrier,barrier随数据流向Writer。以Writer_0为例,以Writer_0接收Reader_0和Reader_1的发来的数据,假设先收到了Reader_0的barrier,这个时候Writer_0停止写出数据到HDFS,将接收到的数据先放到 InputBuffer里面,一直等待Reader_1的barrier到达之后再将Buffer里的数据全部写出,然后生成Writer的Snapshot,整个checkpoint结束后,记录的任务状态为: - -> Reader_0:id=12 -> -> Reader_1:id=11 -> -> Writer_0:id=无法确定 -> -> Writer_1:id=无法确定 - -任务状态会记录到配置的HDFS目录/chunjun/checkpoint/abc123下。因为每个Writer会接收两个Reader的数据,以及各个通道的数据读写速率可能不一样,所以导致writer接收到的数据顺序是不确定的,但是这不影响数据的准确性,因为读取数据时只需要Reader记录的状态就可以构造查询sql,我们只要确保这些数据真的写到HDF就行了。在Writer生成Snapshot之前,会做一系列操作保证接收到的数据全部写入HDFS: - -- close写入HDFS文件的数据流,这时候会在/data_test/.data目录下生成两个两个文件: - - - /data_test/.data/0.abc123.0 - - - /data_test/.data/1.abc123.0 - -- 将生成的两个数据文件移动到/data_test目录下; - -- 更新文件名称模板更新为:channelIndex.abc123.1; - -快照生成后任务继续读写数据,如果生成快照的过程中有任何异常,任务会直接失败,这样这次快照就不会生成,任务恢复时会从上一个成功的快照恢复。 - -##### **4**)任务正常结束 - -任务正常结束时也会做和生成快照时同样的操作,close文件流,移动临时数据文件等。 - -##### **5**)任务异常终止 - -任务如果异常结束,假设任务结束时最后一个checkpoint记录的状态为: - -> Reader_0:id=12 -> -> Reader_1:id=11 - -那么任务恢复的时候就会把各个通道记录的状态赋值给offset,再次读取数据时构造的sql为: - -```sql -第一个通道: -select * from data_test -where id mod 2=0 -and id > 12; - -第二个通道 -select * from data_test -where id mod 2=1 -and id > 11; -``` - -这样就可以从上一次失败的位置继续读取数据了。 - -#### **3.3** **支持断点续传的插件** - -理论上只要支持过滤数据的数据源,和支持事务的数据源都可以支持断点续传的功能,目前ChunJun支持的插件如下: - -| Reader | Writer | -| -------------- | --------------- | -| mysql等关系数据读取插件 | HDFS | -| | FTP | -| | mysql等关系数据库写入插件 | - -### **4.基于binlog的实时采集** - -目前ChunJun支持实时采集的插件有KafKa,binlog插件,binlog插件是专门针对mysql数据库做实时采集的,如果要支持其它的数据源,只需要把数据打到Kafka,然后再用ChunJun的Kafka插件消费数据即可,比如oracle,只需要使用oracle的ogg将数据打到Kafka。这里我们专门讲解一下mysql的实时采集插件binlog。 - -#### **4.1 binlog** - -binlog是Mysql sever层维护的一种二进制日志,与innodb引擎中的redo/undo log是完全不同的日志;其主要是用来记录对mysql数据更新或潜在发生更新的SQL语句,并以"事务"的形式保存在磁盘中。 - -binlog的作用主要有: - -- 复制:MySQL Replication在Master端开启binlog,Master把它的二进制日志传递给slaves并回放来达到master-slave数据一致的目的; - -- 数据恢复:通过mysqlbinlog工具恢复数据; - -- 增量备份; - -#### **4.2 MySQL**主备复制 - -有了记录数据变化的binlog日志还不够,我们还需要借助MySQL的主备复制功能:主备复制是指 一台服务器充当主数据库服务器,另一台或多台服务器充当从数据库服务器,主服务器中的数据自动复制到从服务器之中。 - -
- -
- -主备复制的过程: - -1. MySQL master 将数据变更写入二进制日志( binary log, 其中记录叫做二进制日志事件binary log events,可以通过 show binlog events 进行查看); - -2. MySQL slave 将 master 的 binary log events 拷贝到它的中继日志(relay log); - -3. MySQL slave 重放 relay log 中事件,将数据变更反映它自己的数据; - -#### **4.3 canal** - -有了binlog日志数据和MySQL的主备复制功能,我们只需要模拟一台Slave,将接收到的binlog数据解析出来就可以做到实时采集MySQL的数据变化,阿里巴巴贡献的canal组件就实现了这样的功能。 - -
- -
- -canal工作原理: - -1. canal模拟 MySQL slave 的交互协议,伪装自己为 MySQL slave ,向 MySQL master 发送dump 协议 - -2. MySQL master 收到 dump 请求,开始推送 binary log 给 slave (即 canal ) - -3. canal解析 binary log 对象(原始为 byte 流) - -#### **4.4** **写入Hive** - -binlog插件可以监听多张表的数据变更情况,解析出的数据中包含表名称信息,读取到的数据可以全部写入目标数据库的一张表,也可以根据数据中包含的表名信息写入不同的表,目前只有Hive插件支持这个功能。Hive插件目前只有写入插件,功能基于HDFS的写入插件实现,也就是说从binlog读取,写入hive也支持失败恢复的功能。 - -
- -
- -写入Hive的过程: - -1. 从数据中解析出MySQL的表名,然后根据表名映射规则转换成对应的Hive表名; - -2. 检查Hive表是否存在,如果不存在就创建Hive表; - -3. 查询Hive表的相关信息,构造HdfsOutputFormat; - -4. 调用HdfsOutputFormat将数据写入HDFS; diff --git a/docs/statistics.md b/docs/statistics.md deleted file mode 100644 index e142815a92..0000000000 --- a/docs/statistics.md +++ /dev/null @@ -1,161 +0,0 @@ -ChunJun使用了flink内置Accumulator和Metric来记录任务的一些统计指标: - -| 指标名称 | 含义 | -| ---------------- | ----------- | -| numRead | 累计读取数据条数 | -| byteRead | 累计读取数据字节数 | -| readDuration | 读取数据的总时间 | -| | | -| numWrite | 累计写入数据条数 | -| byteWrite | 累计写入数据字节数 | -| writeDuration | 写入数据的总时间 | -| nErrors | 累计错误记录数 | -| nullErrors | 累计空指针错误记录数 | -| duplicateErrors | 累计主键冲突错误记录数 | -| conversionErrors | 累计类型转换错误记录数 | -| otherErrors | 累计其它错误记录数 | - -### 获取统计指标的方式 - -#### 1.Local模式运行 - -local模式运行时,任务结束后会在控制台打印这些指标: - -``` ---------------------------------- -numWrite | 100 -last_write_num_0 | 0 -conversionErrors | 0 -writeDuration | 12251 -numRead | 100 -duplicateErrors | 0 -snapshotWrite | 0 -readDuration | 12247 -otherErrors | 0 -byteRead | 2329 -last_write_location_0 | 0 -byteWrite | 2329 -nullErrors | 0 -nErrors | 0 ---------------------------------- -``` - -#### 2.yarn模式运行 - -##### 2.1 通过Flink REST接口获取 - -任务运行期间,可以通过Flink REST接口获取Accumulator数据,名称和上面给出的一致。 - -api:http://host:8088/proxy/application_1569335225689_4172//jobs/d5582272d29ff38e10416a4043a86cad/accumulators - -返回数据示例: - -```json -{ - "job-accumulators": [], - "user-task-accumulators": [ - { - "name": "numWrite", - "type": "LongCounter", - "value": "0" - }, - { - "name": "last_write_num_0", - "type": "LongCounter", - "value": "0" - }, - { - "name": "conversionErrors", - "type": "LongCounter", - "value": "0" - }, - { - "name": "writeDuration", - "type": "LongCounter", - "value": "0" - }, - { - "name": "numRead", - "type": "LongCounter", - "value": "0" - }, - { - "name": "duplicateErrors", - "type": "LongCounter", - "value": "0" - }, - { - "name": "snapshotWrite", - "type": "LongCounter", - "value": "0" - }, - { - "name": "readDuration", - "type": "LongCounter", - "value": "0" - }, - { - "name": "otherErrors", - "type": "LongCounter", - "value": "0" - }, - { - "name": "byteRead", - "type": "LongCounter", - "value": "0" - }, - { - "name": "last_write_location_0", - "type": "LongCounter", - "value": "0" - }, - { - "name": "byteWrite", - "type": "LongCounter", - "value": "0" - }, - { - "name": "nullErrors", - "type": "LongCounter", - "value": "0" - }, - { - "name": "nErrors", - "type": "LongCounter", - "value": "0" - } - ], - "serialized-user-task-accumulators": {} -} -``` - -##### 2.2 将指标输出到其它系统 - -比如将指标输出到prometheus,在flink的配置文件里增加配置即可: - -``` -metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter -metrics.reporter.promgateway.interval: 500 MILLISECONDS -metrics.reporter.promgateway.host: 127.0.0.1 -metrics.reporter.promgateway.port: 9091 -metrics.reporter.promgateway.jobName: testjob -metrics.reporter.promgateway.randomJobNameSuffix: true -metrics.reporter.promgateway.deleteOnShutdown: false -``` - -通过prometheus获取数据时的名称为: - -| ChunJun中指标名称 | prometheus中指标名称 | -| ---------------- | ----------------------------------------------------------- | -| numRead | flink_taskmanager_job_task_operator_chunjun_byteRead | -| byteRead | flink_taskmanager_job_task_operator_chunjun_byteRead | -| readDuration | flink_taskmanager_job_task_operator_chunjun_readDuration | -| | | -| numWrite | flink_taskmanager_job_task_operator_chunjun_numWrite | -| byteWrite | flink_taskmanager_job_task_operator_chunjun_byteWrite | -| writeDuration | flink_taskmanager_job_task_operator_chunjun_writeDuration | -| nErrors | flink_taskmanager_job_task_operator_chunjun_nErrors | -| nullErrors | flink_taskmanager_job_task_operator_chunjun_nullErrors | -| duplicateErrors | flink_taskmanager_job_task_operator_chunjun_duplicateErrors | -| conversionErrors | flink_taskmanager_job_task_operator_chunjun_conversionErrors | -| otherErrors | flink_taskmanager_job_task_operator_chunjun_otherErrors | diff --git "a/docs_zh/ChunJun \351\200\232\347\224\250\351\205\215\347\275\256\350\257\246\350\247\243.md" "b/docs_zh/ChunJun \351\200\232\347\224\250\351\205\215\347\275\256\350\257\246\350\247\243.md" new file mode 100644 index 0000000000..9d98a349d7 --- /dev/null +++ "b/docs_zh/ChunJun \351\200\232\347\224\250\351\205\215\347\275\256\350\257\246\350\247\243.md" @@ -0,0 +1,253 @@ +# ChunJun 通用配置详解 + +## 配置文件结构详解 + +一个完整的 ChunJun 任务脚本配置包含 content, setting 两个部分。content 用于配置任务的输入源与输出源,其中包含 reader,writer。而 setting 则配置任务整体的环境设定,其中包含 speed,errorLimit,metricPluginConf,restore,log,dirty。总体结构如下所示: + +```JSON +{ + "job" : { + "content" :[{ + "reader" : {}, + "writer" : {} + }], + "setting" : { + "speed" : {}, + "errorLimit" : {}, + "metricPluginConf" : {}, + "restore" : {}, + "log" : {}, + "dirty":{} + } + } +} +``` + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称说明是否必填
contendreaderreader插件详细配置
writerwriter插件详细配置
settingspeed速率限制
errorLimit出错控制
metricPluginConf指标插件配置
restore任务类型及断点续传配置
log日志记录配置
dirty脏数据保存
+ +## Content 配置 + +### Reader + +reader 用于配置数据的输入源,即数据从何而来。具体配置如下所示: + +```json +"reader" : { + "name" : "xxreader", + "parameter" : { + ...... + } +} +``` + +| 名称 | 说明 | 是否必填 | +| --------- | --------------------------------------------- | -------- | +| name | reader 插件名称,具体名称参考各数据源配置文档 | 是 | +| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | + +### Writer + +```JSON +"writer" : { + "name" : "xxwriter", + "parameter" : { + ...... + } +} +``` + +| 名称 | 说明 | 是否必填 | +| --------- | --------------------------------------------- | -------- | +| name | writer 插件名称,具体名称参考各数据源配置文档 | 是 | +| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | + +## Setting 配置 + +### speed + +speed 用于配置任务并发数及速率限制。具体配置如下所示: + +```json +"speed" : { + "channel": 1, + "readerChannel": -1, + "writerChannel": -1, + "bytes": 0, + "rebalance" : true +} +``` + +| 名称 | 说明 | 是否必填 | 默认值 | 数据类型 | +| ------------- | -------------------------------------- | -------- | ------ | -------- | +| channel | 任务并发数 | 否 | 1 | Integer | +| readerChannel | source 并行度,-1 代表采用全局并行度 | 否 | -1 | Integer | +| writerChannel | sink 并行度,-1 代表采用全局并行度 | 否 | -1 | Integer | +| bytes | bytes >0 则表示开启任务限速 | 否 | 0 | Long | +| rebalance | 是否强制进行 rebalance,开启会消耗性能 | 否 | false | Boolean | + +### ErrorLimit + +errorLimit 用于配置任务运行时数据读取写入的出错控制。具体配置如下所示: + +```json +"errorLimit" : { + "record": 100, + "percentage": 10.0 +} +``` + +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| ---------- | ------------------------------------------------ | -------- | ------ | -------- | +| record | 错误阈值,当错误记录数超过此阈值时任务失败 | 否 | 0 | Integer | +| percentage | 错误比例阈值,当错误记录比例超过此阈值时任务失败 | 否 | 0.0 | Double | + +### MetricPluginConf + +metricPluginConf 用于配置 chunjun 指标相关信息。 + +目前只应用于 Jdbc 插件中,在作业结束时将 StartLocation 和 EndLocation 指标发送到指定数据源中。 + +目前支持 Prometheus 和 Mysql。具体配置如下所示: + +#### Prometheus + +promethusReporter 依赖 pushGateway 和普罗米修斯交互 + +```json +"metricPluginConf" : { + "pluginName": "promethus" +} +``` + +普罗米修斯的相关配置信息需要配置在 flink 的配置文件中 + +```yaml +metrics.reporter.promgateway.host: 127.0.0.1 +metrics.reporter.promgateway.port: 9091 +metrics.reporter.promgateway.jobName: testjob +metrics.reporter.promgateway.randomJobNameSuffix: true +metrics.reporter.promgateway.deleteOnShutdown: false +``` + +| 名称 | 说明 | 是否必填 | 默认值 | +| ------------------------------------------------ | ---------------------------------------- | -------- | ------ | +| metrics.reporter.promgateway.host | pushGateway 域名 | 是 | 无 | +| metrics.reporter.promgateway.port | pushGateway 端口 | 是 | 0 | +| metrics.reporter.promgateway.jobName | 指标对应作业名 | 否 | 无 | +| metrics.reporter.promgateway.randomJobNameSuffix | 是否对作业名加上随机后缀,防止作业名重复 | 否 | false | +| metrics.reporter.promgateway.deleteOnShutdown | 是否在作业结束后删除指标信息 | 否 | true | + +#### Mysql + +要求目标表存在至少两个 String 字段,metric_name 和 metric_value,分别记录指标名和指标值 + +```json +"metricPluginConf" : { + "pluginName": "promethus" + "pluginProp": { + "jdbcUrl":"", + "database":"", + "table":"", + "username":"", + "password":"", + "properties":{ + } + } +} +``` + +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| ---------- | ----------------------------- | -------- | ------ | -------- | +| jdbcUrl | 针对 Mysql 的 jdbc 连接字符串 | 是 | 无 | String | +| table | mysql 表名 | 是 | 无 | String | +| username | mysql 用户名 | 是 | 无 | String | +| password | mysql 用户名对应的密码 | 是 | 无 | String | +| properties | 数据库连接配置 | 否 | 无 | Map | + +### Restore + +restore 用于配置同步任务类型(离线同步、实时采集)和断点续传功能。具体配置如下所示: + +```json +"restore" : { + "isStream" : false, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 +} +``` + +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| ------------------ | ------------------- | ------------------ | ------ | -------- | +| isStream | 是否为实时采集任务 | 否 | false | Boolean | +| isRestore | 是否开启断点续传 | 否 | false | Boolean | +| restoreColumnName | 断点续传字段名称 | 开启断点续传后必填 | 无 | String | +| restoreColumnIndex | 断点续传字段索引 ID | 开启断点续传后必填 | 无 | Integer | + +### Log + +log 用于配置 ChunJun 中定义的插件日志的保存与记录。具体配置如下所示: + +```json +"log" : { + "isLogger": false, + "level" : "info", + "path" : "/tmp/dtstack/chunjun/", + "pattern":"" +} +``` + +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| -------- | -------------------- | -------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------- | -------- | +| isLogger | 是否保存日志记录 | 否 | false | Boolean | +| level | 日志级别 | 否 | info | String | +| path | 服务器上日志保存路径 | 否 | /tmp/dtstack/chunjun/ | String | +| pattern | 日志输出格式 | 否 | log4j:%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n
logback : %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n | String | diff --git "a/docs_zh/ChunJun\346\213\223\345\261\225\346\225\260\346\215\256\346\240\274\345\274\217/protobuf-x.md" "b/docs_zh/ChunJun\346\213\223\345\261\225\346\225\260\346\215\256\346\240\274\345\274\217/protobuf-x.md" new file mode 100644 index 0000000000..d4057aa64c --- /dev/null +++ "b/docs_zh/ChunJun\346\213\223\345\261\225\346\225\260\346\215\256\346\240\274\345\274\217/protobuf-x.md" @@ -0,0 +1,203 @@ +# ProtoBuf Format + +## 一、介绍 + +protobuf format允许读写protocol buffer数据,目前仅支持sql + +## 二、支持版本 + +protocol buffer2.x、protocol buffer3.x + + +## 三、format名称 + +protobuf-x + +## 四、如何写一个protobuf format相关的DDL + +```sql +CREATE TABLE proto ( + user_id BIGINT, + item_id BIGINT, + category_id BIGINT, + behavior STRING, + ts BIGINT +) WITH ( + 'connector' = 'kafka', + 'topic' = 'user_behavior', + 'properties.bootstrap.servers' = 'localhost:9092', + 'properties.group.id' = 'testGroup', + 'format' = 'protobuf-x', + 'protobuf-x.message-class-name' = 'com.dtstack.chunjun.behavior.behaviorOuterClass$Message' + ) +``` + +## 五、format配置项 + +| 配置项 | 是否必填 | 默认值 | 数据类型 | 描述 | +| ----------------------------- | -------- | ------ | -------- | ------------------------------------------------------------ | +| protobuf-x.message-class-name | 是 | 无 | String | protobuf消息体在proto文件生成的outerClass类的中对应的类的全路径 | + +## 六、数据类型映射 + +| Flink SQL type | ProtobufType | +| ------------------------------------ | ------------------------------------------------------------ | +| STRINGSTRING | string、enumstring、enum | +| BOOLEANBOOLEAN | boolbool | +| BINARY / VARBINARYBINARY / VARBINARY | | +| BYTES | bytes | +| DECIMAL | | +| TINYINT | | +| SMALLINT | | +| INT | int32、uint32、sint32、fixedint32、sfixed32 | +| BIGINT | int64、uint64、sint64、fixedint64、sfixed64 | +| FLOAT | float | +| DOUBLE | double | +| DATE | | +| TIME | | +| TIMESTAMP | | +| ARRAY | repeated修饰符 | +| MAP | map(key,只能是除了浮点型和字节之外的标量类型;value,不能是map)https://developers.google.com/protocol-buffers/docs/proto3#maps | +| MULTISET | | +| ROW | nested,oneOf(Row的第一项为case) | + +## Example + +### proto文件 + +```protobuf +syntax = "proto3"; + +package ZPMC.Message; + +message RepeatedBool {repeated bool Values = 1;} + +message RepeatedMyBool {repeated bool Values = 1;} + +message RepeatedInt32 {repeated int32 Values = 1;} + +message RepeatedUint32 {repeated uint32 Values = 1;} + +message RepeatedInt64 {repeated int64 Values = 1;} + +message RepeatedUint64 {repeated uint64 Values = 1;} + +message RepeatedFloat {repeated float Values = 1;} + +message RepeatedDouble {repeated double Values = 1;} + +message RepeatedString {repeated string Values = 1;} + +message Variant{ + oneof Value{ + bool ValueBool = 1; + RepeatedBool ArrayBool = 2; + int32 ValueInt32 = 3; + RepeatedInt32 ArrayInt32 = 4; + uint32 ValueUint32 = 5; + RepeatedUint32 ArrayUint32 = 6; + int64 ValueInt64 = 7; + RepeatedInt64 ArrayInt64 = 8; + uint64 ValueUint64 = 9; + RepeatedUint64 ArrayUint64 = 10; + float ValueFloat = 11; + RepeatedFloat ArrayFloat = 12; + double ValueDouble = 13; + RepeatedDouble arrayDouble=14; + string ValueString = 15; + RepeatedString ArrayString = 16; + bytes ValueBytes = 17; + int64 ValueTimestamp = 18; + }; + bool boolx = 19; + oneof Value2{ + bool ValueBool2 = 20; + RepeatedBool ArrayBool2 = 21; + } + bool booly=22; +} + +message MessageItem{ + string TagName = 1; //默认optional + Variant TagValue = 2; + int32 UaDataType = 3; + bool Quality = 4; + int64 Timestamp = 5; + mapTagInfos = 6; + mapExValues = 7; +} + +message MessageGroup{ + map GroupInfo = 1; + repeated MessageItem Messages = 2; +} + +``` + +### DDL + +```sql +CREATE TABLE reader ( + GroupInfo MAP + ,Messages ARRAY< + ROW< + TAGNAME VARCHAR , + TagValue ROW< + `Value` ROW< + ValueCase INTEGER, + ValueBool BOOlEAN, + ArrayBool ROW<`Values` ARRAY>, + ValueInt32 INTEGER, + ArrayInt32 ROW<`Values` ARRAY>, + ValueUint32 INTEGER, + ArrayUint32 ROW<`Values` ARRAY>, + ValueInt64 BIGINT, + ArrayInt64 ROW<`Values` ARRAY>, + ValueUint64 BIGINT, + ArrayUint64 ROW<`Values` ARRAY>, + ValueFloat FLOAT, + ArrayFloat ROW<`Values` ARRAY>, + ValueDouble DOUBLE, + ArrayDouble ROW<`Values` ARRAY>, + ValueString STRING, + ArrayString ROW<`Values` ARRAY>, + ValueBytes BINARY, + ValueTimestamp BIGINT + >, + boolx BOOLEAN, + Value2 ROW< + ValueCase INTEGER , + Value2Value BOOLEAN, + ArrayBool2 ROW<`Values` ARRAY> + >, + booly BOOLEAN + >, + UaDataType INTEGER, + Quality BOOLEAN, + `Timestamp` BIGINT, + TagInfos MAP, + ExValues MAP + > + > + + + +-- , `topic` STRING METADATA VIRTUAL -- from Kafka connector +-- , `leader-epoch` int METADATA VIRTUAL -- from Kafka connector +-- , `offset` BIGINT METADATA VIRTUAL -- from Kafka connector +-- , ts TIMESTAMP(3) METADATA FROM 'timestamp' -- from Kafka connector +-- , `timestamp-type` STRING METADATA VIRTUAL -- from Kafka connector +-- , partition_id BIGINT METADATA FROM 'partition' VIRTUAL -- from Kafka connector + +) WITH ( + 'connector' = 'kafka-x' + ,'topic' = 'liuliu_proto_source' + ,'properties.bootstrap.servers' = 'flink01:9092' + ,'properties.group.id' = 'luna_g' + ,'scan.startup.mode' = 'earliest-offset' + ,'format' = 'protobuf-x' + ,'protobuf-x.message-class-name' = 'ZPMC.Message.MessageGroupOuterClass$MessageGroup' + ,'scan.parallelism' = '1' +); +``` + diff --git a/docs/connectors/binlog/binlog-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/binlog/binlog-source.md" similarity index 82% rename from docs/connectors/binlog/binlog-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/binlog/binlog-source.md" index 567f1bf077..645c02de0a 100644 --- a/docs/connectors/binlog/binlog-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/binlog/binlog-source.md" @@ -1,126 +1,119 @@ -# MySQL Binlog Source - +# Binlog Source -- [一、介绍](#一介绍) -- [二、支持版本](#二支持版本) -- [三、插件名称](#三插件名称) -- [四、数据库配置](#四数据库配置) - - [1 、修改配置文件](#1修改配置文件) - - [2 、添加权限](#2添加权限) -- [五、参数说明](#五参数说明) - - [1 、Sync](#1sync) - - [2 、SQL](#2sql) -- [六、数据结构](#六数据结构) -- [七、数据类型](#七数据类型) -- [八、脚本示例](#八脚本示例) - - - -## 一、介绍 -MySQL Binlog插件使用Canal组件实时地从MySQL中捕获变更数据。目前sink插件暂不支持数据还原,只能写入变更的日志数据。 - -## 二、支持版本 +## 一、介绍 + +Binlog插件使用Canal组件实时地从MySQL中捕获变更数据。source同步插件支持数据还原,支持DDL的变更。 + +## 二、支持版本 + MySQL 5.1.5及以上、TiDB 3.0.10之后 ## 三、插件名称 + | Sync | binlogsource、binlogreader | | --- | --- | | SQL | binlog-x | +## 四、数据库配置 +### 1、修改配置文件 -## 四、数据库配置 -### 1、修改配置文件 binlog_format需要修改为 ROW 格式,在/etc/my.cnf文件里[mysqld]下添加下列配置 + ```sql -server_id=109 +server_id +=109 log_bin = /var/lib/mysql/mysql-bin binlog_format = ROW expire_logs_days = 30 ``` +### 2、添加权限 -### 2、添加权限 MySQL Binlog权限需要三个权限 SELECT, REPLICATION SLAVE, REPLICATION CLIENT + ```sql -GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal'; +GRANT +SELECT, REPLICATION SLAVE, REPLICATION CLIENT +ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal'; ``` - - 缺乏SELECT权限时,报错为 + ```text com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: Access denied for user 'canal'@'%' to database 'binlog' ``` - 缺乏REPLICATION SLAVE权限时,报错为 + ```text java.io.IOException: Error When doing Register slave:ErrorPacket [errorNumber=1045, fieldCount=-1, message=Access denied for user 'canal'@'%' ``` - 缺乏REPLICATION CLIENT权限时,报错为 + ```text com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: Access denied; you need (at least one of) the SUPER, REPLICATION CLIENT privilege(s) for this operation ``` - Binlog为什么需要这些权限: - Select权限代表允许从表中查看数据 - Replication client权限代表允许执行show master status,show slave status,show binary logs命令 - Replication slave权限代表允许slave主机通过此用户连接master以便建立主从 复制关系 +## 五、参数说明 - -## 五、参数说明 -### 1、Sync +### 1、Sync - **jdbcUrl** - - 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + - + 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - 必选:是 - 字段类型:string - 默认值:无 -
+ - **username** - 描述:数据源的用户名 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **password** - 描述:数据源指定用户名的密码 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **host** - 描述:启动MySQL slave的机器ip - 必选:是 - 字段类型:string - 默认值:无 -
+ - **port** - 描述:启动MySQL slave的端口 - 必选:否 - 字段类型:int - 默认值:3306 -
+ - **table** - 描述:需要解析的数据表。 - 注意:指定此参数后filter参数将无效,table和filter都为空,监听jdbcUrl里的schema下所有表 - 必选:否 - - 字段类型:list + - 字段类型:list\ - 默认值:无 -
+ - **filter** - 描述:过滤表名的Perl正则表达式 @@ -133,7 +126,7 @@ Binlog为什么需要这些权限: - canal schema下所有表: `canal\\..*` - canal下的以canal打头的表:`canal\\.canal.*` - canal schema下的一张表:`canal.test1` -
+ - **cat** - 描述:需要解析的数据更新类型,包括insert、update、delete三种 @@ -141,7 +134,7 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:string - 默认值:无 -
+ - **start** - 描述:要读取的binlog文件的开始位置 @@ -152,34 +145,28 @@ Binlog为什么需要这些权限: - position:文件的指定位置,采集起点从指定文件的指定位置处消费 - 字段类型:map - 默认值:无 -
-- **transactionSize** - - 描述:DML数据缓存大小。 - - 注意: - - 必选:否 - - 默认值:1024 - **pavingData** - 描述:是否将解析出的json数据拍平,具体见[六、数据结构](#六数据结构) - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **splitUpdate** - 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见[六、数据结构](#六数据结构) - 必选:否 - 字段类型:boolean - 默认值:false -
+ - **timestampFormat** - 描述:指定输入输出所使用的timestamp格式,可选值:`SQL`、`ISO_8601` - 必选:否 - 字段类型:string - 默认值:SQL -
+ - **slaveId** - 描述:从服务器的ID @@ -187,28 +174,27 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:long - 默认值:new Object().hashCode() -
+ - **connectionCharset** - 描述:编码信息 - 必选:否 - 字段类型:string - 默认值:UTF-8 -
+ - **detectingEnable** - 描述:是否开启心跳 - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **detectingSQL** - 描述:心跳SQL - 必选:否 - 字段类型:string - 默认值:SELECT CURRENT_DATE -
- **enableTsdb** @@ -216,21 +202,21 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **bufferSize** - 描述:并发缓存大小 - 注意:必须为2的幂 - 必选:否 - 默认值:1024 -
+ - **parallel** - 描述:是否开启并行解析binlog日志 - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **parallelThreadSize** - 描述:并行解析binlog日志线程数 @@ -238,64 +224,64 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:int - 默认值:2 -
+ - **isGTIDMode** - 描述:是否开启gtid模式 - 必选:否 - 字段类型:boolean - 默认值:false -
+ - **queryTimeOut** - 描述:通过TCP连接发送数据(在这里就是要执行的sql)后,等待响应的超时时间,单位毫秒 - 必选:否 - 字段类型:int - 默认值:300000 -
+ - **connectTimeOut** - 描述:数据库驱动(mysql-connector-java)与mysql服务器建立TCP连接的超时时间,单位毫秒 - 必选:否 - 字段类型:int - 默认值:60000 -
-### 2、SQL +### 2、SQL + - **url** - 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - 必选:是 - 字段类型:string - 默认值:无 -
+ - **username** - 描述:数据源的用户名 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **password** - 描述:数据源指定用户名的密码 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **host** - 描述:启动MySQL slave的机器ip - 必选:是 - 字段类型:string - 默认值:无 -
+ - **port** - 描述:启动MySQL slave的端口 - 必选:否 - 字段类型:int - 默认值:3306 -
+ - **table** - 描述:需要解析的数据表。 @@ -303,7 +289,7 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:string - 默认值:无 -
+ - **filter** - 描述:过滤表名的Perl正则表达式 @@ -312,7 +298,7 @@ Binlog为什么需要这些权限: - 字段类型:string - 默认值:无 - 例子:canal schema下的一张表:`canal.test1` -
+ - **cat** - 描述:需要解析的数据更新类型,包括insert、update、delete三种 @@ -320,70 +306,70 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:string - 默认值:无 -
+ - **timestamp** - 描述:要读取的binlog文件的开始位置,时间戳,采集起点从指定的时间戳处消费; - 必选:否 - 字段类型:string - 默认值:无 -
+ - **journal-name** - 描述:要读取的binlog文件的开始位置,文件名,采集起点从指定文件的起始处消费; - 必选:否 - 字段类型:string - 默认值:无 -
+ - **position** - 描述:要读取的binlog文件的开始位置,文件的指定位置,采集起点从指定文件的指定位置处消费 - 必选:否 - 字段类型:string - 默认值:无 -
+ - **connection-charset** - 描述:编码信息 - 必选:否 - 字段类型:string - 默认值:UTF-8 -
+ - **detecting-enable** - 描述:是否开启心跳 - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **detecting-sql** - 描述:心跳SQL - 必选:否 - 字段类型:string - 默认值:SELECT CURRENT_DATE -
+ - **enable-tsdb** - 描述:是否开启时序表结构能力 - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **buffer-size** - 描述:并发缓存大小 - 注意:必须为2的幂 - 必选:否 - 默认值:1024 -
+ - **parallel** - 描述:是否开启并行解析binlog日志 - 必选:否 - 字段类型:boolean - 默认值:true -
+ - **parallel-thread-size** - 描述:并行解析binlog日志线程数 @@ -391,28 +377,28 @@ Binlog为什么需要这些权限: - 必选:否 - 字段类型:int - 默认值:2 -
+ - **is-gtid-mode** - 描述:是否开启gtid模式 - 必选:否 - 字段类型:boolean - 默认值:false -
+ - **query-time-out** - 描述:通过TCP连接发送数据(在这里就是要执行的sql)后,等待响应的超时时间,单位毫秒 - 必选:否 - 字段类型:int - 默认值:300000 -
+ - **connect-time-out** - 描述:数据库驱动(mysql-connector-java)与mysql服务器建立TCP连接的超时时间,单位毫秒 - 必选:否 - 字段类型:int - 默认值:60000 -
+ - **timestamp-format.standard** - 描述:同Sync中的`timestampFormat`参数,指定输入输出所使用的timestamp格式,可选值:`SQL`、`ISO_8601` @@ -420,37 +406,57 @@ Binlog为什么需要这些权限: - 字段类型:string - 默认值:SQL -## 六、数据结构 +## 六、数据结构 + 在2020-01-01 12:30:00(时间戳:1577853000000)执行: + ```sql -INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) VALUES (1, 1, 'a'); +INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) +VALUES (1, 1, 'a'); ``` + 在2020-01-01 12:31:00(时间戳:1577853060000)执行: + ```sql -DELETE FROM `tudou`.`kudu` WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; +DELETE +FROM `tudou`.`kudu` +WHERE `id` = 1 + AND `user_id` = 1 + AND `name` = 'a'; ``` + 在2020-01-01 12:32:00(时间戳:1577853180000)执行: + ```sql -UPDATE `tudou`.`kudu` SET `id` = 2, `user_id` = 2, `name` = 'b' WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a'; +UPDATE `tudou`.`kudu` +SET `id` = 2, + `user_id` = 2, + `name` = 'b' +WHERE `id` = 1 + AND `user_id` = 1 + AND `name` = 'a'; ``` -1、pavingData = true, splitUpdate = false -RowData中的数据依次为: + +1、pavingData = true, splitUpdate = false RowData中的数据依次为: + ``` //schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name ["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] ["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] ["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", 1, 1, "a", 2, 2, "b"] ``` -2、pavingData = false, splitUpdate = false -RowData中的数据依次为: + +2、pavingData = false, splitUpdate = false RowData中的数据依次为: + ``` //schema, table, ts, opTime, type, before, after ["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] ["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] ["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", {"id":1, "user_id":1, "name":"a"}, {"id":2, "user_id":2, "name":"b"}] ``` -3、pavingData = true, splitUpdate = true -RowData中的数据依次为: + +3、pavingData = true, splitUpdate = true RowData中的数据依次为: + ``` //schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name ["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] @@ -462,8 +468,9 @@ RowData中的数据依次为: //schema, table, ts, opTime, type, after_id, after_user_id, after_name ["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", 2, 2, "b"] ``` -4、pavingData = false, splitUpdate = true -RowData中的数据依次为: + +4、pavingData = false, splitUpdate = true RowData中的数据依次为: + ``` //schema, table, ts, opTime, type, before, after ["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] @@ -477,14 +484,16 @@ RowData中的数据依次为: - type:变更类型,INSERT,UPDATE、DELETE - opTime:数据库中SQL的执行时间 - ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下: + ```java -long id = Long.parseLong("6760525407742726144"); -long res = id >> 22; -DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -System.out.println(sdf.format(res)); //2021-01-28 19:54:21 +long id=Long.parseLong("6760525407742726144"); + long res=id>>22; + DateFormat sdf=new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + System.out.println(sdf.format(res)); //2021-01-28 19:54:21 ``` -## 七、数据类型 +## 七、数据类型 + | 支持 | BIT | | --- | --- | | | TINYINT、SMALLINT、MEDIUMINT、INT、INT24、INTEGER、FLOAT、DOUBLE、REAL、LONG、BIGINT、DECIMAL、NUMERIC | @@ -493,6 +502,6 @@ System.out.println(sdf.format(res)); //2021-01-28 19:54:21 | | TINYBLOB、BLOB、MEDIUMBLOB、LONGBLOB、GEOMETRY、BINARY、VARBINARY | | 暂不支持 | 无 | +## 八、脚本示例 -## 八、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/clickhouse/clickhouse-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-lookup.md" similarity index 81% rename from docs/connectors/clickhouse/clickhouse-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-lookup.md" index dd17a9b310..b6089c85b6 100644 --- a/docs/connectors/clickhouse/clickhouse-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-lookup.md" @@ -28,6 +28,13 @@ ClickHouse 19.x及以上 - 字段类型:String - 默认值:无 +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ ​
- **table-name** @@ -111,26 +118,13 @@ ClickHouse 19.x及以上 ​ ## 五、数据类型 -| 支持 | BOOLEAN | -| --- | --- | -| | TINYINT | -| | SMALLINT | -| | INT | -| | BIGINT | -| | FLOAT | -| | DOUBLE | -| | DECIMAL | -| | STRING | -| | VARCHAR | -| | CHAR | -| | TIMESTAMP | -| | DATE | -| | BINARY | -| | NULL | -| 暂不支持 | ARRAY | -| | MAP | -| | STRUCT | -| | UNION | + +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、TIMESTAMP、DATE、BINARY、NULL | +| 不支持 | ARRAY、MAP、STRUCT、UNION | + + ## 六、配置示例 diff --git a/docs/connectors/clickhouse/clickhouse-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-sink.md" similarity index 90% rename from docs/connectors/clickhouse/clickhouse-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-sink.md" index 87be380a68..19d0eaad14 100644 --- a/docs/connectors/clickhouse/clickhouse-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-sink.md" @@ -151,6 +151,13 @@ ClickHouse 19.x及以上 ​
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -226,28 +233,13 @@ ClickHouse 19.x及以上 ## 五、数据类型 -| 支持 | BOOLEAN | -| --- | --- | -| | TINYINT | -| | SMALLINT | -| | INT | -| | BIGINT | -| | FLOAT | -| | DOUBLE | -| | DECIMAL | -| | STRING | -| | VARCHAR | -| | CHAR | -| | TIMESTAMP | -| | DATE | -| | BINARY | -| | NULL | -| 暂不支持 | ARRAY | -| | MAP | -| | STRUCT | -| | UNION | +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、TIMESTAMP、DATE、BINARY、NULL | +| 不支持 | ARRAY、MAP、STRUCT、UNION | + ## 六、配置示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/clickhouse/clickhouse-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-source.md" similarity index 94% rename from docs/connectors/clickhouse/clickhouse-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-source.md" index 96837afdb2..2d16d9b8cc 100644 --- a/docs/connectors/clickhouse/clickhouse-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/clickhouse/clickhouse-source.md" @@ -217,6 +217,13 @@ ClickHouse 19.x及以上 ​
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -337,27 +344,11 @@ ClickHouse 19.x及以上 ## 五、数据类型 -| 支持 | BOOLEAN | -| --- | --- | -| | TINYINT | -| | SMALLINT | -| | INT | -| | BIGINT | -| | FLOAT | -| | DOUBLE | -| | DECIMAL | -| | STRING | -| | VARCHAR | -| | CHAR | -| | TIMESTAMP | -| | DATE | -| | BINARY | -| | NULL | -| 暂不支持 | ARRAY | -| | MAP | -| | STRUCT | -| | UNION | +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、TIMESTAMP、DATE、BINARY、NULL | +| 不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、配置示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/db2/db2-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-lookup.md" similarity index 90% rename from docs/connectors/db2/db2-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-lookup.md" index 052ca946ef..32c1e2e496 100644 --- a/docs/connectors/db2/db2-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-lookup.md" @@ -22,12 +22,19 @@ db2维表,支持全量和异步方式

- **url** - - 描述:jdbc:db2://localhost:50002/DT_TEST + - 描述:jdbc:db2://hostname:port/test - 必选:是 - 参数类型:String - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -92,9 +99,10 @@ db2维表,支持全量和异步方式

## 五、数据类型 -| 支持 | INT,BIGINT,SMALLINT,DOUBLE,DECFLOAT,DECIMAL,VARCHAR,CHAR,CLOB,DECIMAL,TIMESTAMP,DATETIME,DATE,TIME,BYTES| +| 是否支持 | 类型名称| | --- | --- | -| 暂不支持 | | +| 支持 | INT,BIGINT,SMALLINT,DOUBLE,DECFLOAT,DECIMAL,VARCHAR,CHAR,CLOB,DECIMAL,TIMESTAMP,DATETIME,DATE,TIME,BYTES| +| 不支持 | | ## 六、脚本示例 diff --git a/docs/connectors/db2/db2-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-sink.md" similarity index 94% rename from docs/connectors/db2/db2-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-sink.md" index 15be9f8257..e1782940ad 100644 --- a/docs/connectors/db2/db2-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-sink.md" @@ -1,4 +1,4 @@ -# db2 Sink +# DB2 Sink ## 一、介绍 db2 sink @@ -21,7 +21,7 @@ db2 sink - 默认值:无 ```text "connection": [{ - "jdbcUrl": ["jdbc:db2://localhost:50002/DT_TEST"], + "jdbcUrl": ["jdbc:db2://hostname:port/test"], "table": ["table"], "schema":"public" }] @@ -137,12 +137,19 @@ db2 sink
- **url** - - 描述:jdbc:db2://localhost:50002/DT_TEST + - 描述:jdbc:db2://hostname:port/test - 必选:是 - 参数类型:String - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -204,9 +211,10 @@ db2 sink
## 五、数据类型 -| 支持 | INT,BIGINT,SMALLINT,DOUBLE,DECFLOAT,DECIMAL,VARCHAR,CHAR,CLOB,DECIMAL,TIMESTAMP,DATETIME,DATE,TIME,BYTES| +| 是否支持 | 类型名称| | --- | --- | -| 暂不支持 | | +| 支持 | INT,BIGINT,SMALLINT,DOUBLE,DECFLOAT,DECIMAL,VARCHAR,CHAR,CLOB,DECIMAL,TIMESTAMP,DATETIME,DATE,TIME,BYTES| +| 不支持 | | ## 六、脚本示例 diff --git a/docs/connectors/db2/db2-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-source.md" similarity index 90% rename from docs/connectors/db2/db2-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-source.md" index f1a8cb8c50..bc35a61c77 100644 --- a/docs/connectors/db2/db2-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/db2/db2-source.md" @@ -20,9 +20,9 @@ - 必选:是 - 参数类型:List - 默认值:无 - ```text + ```json "connection": [{ - "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], + "jdbcUrl": ["jdbc:db2://hostname:port/test"], "table": ["table"], "schema":"public" }] @@ -113,24 +113,25 @@ - **column** - 描述:需要读取的字段。 - - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash - "column":["*"] - ``` - 2.只指定字段名称: - ``` - "column":["id","name"] - ``` - 3.指定具体信息: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` + - 格式:支持3种格式 + - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + ```json + "column":["*"] + ``` + - 2.只指定字段名称: + ```json + "column":["id","name"] + ``` + - 3.指定具体信息: + + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` - 属性说明: - name:字段名称 - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 @@ -150,14 +151,14 @@ - **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 - - 参数类型:long + - 参数类型:Long - 默认值:5000
- **increColumn** - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) - 必选:否 - - 参数类型:String或int + - 参数类型:String或Int - 默认值:无
@@ -178,25 +179,32 @@ - **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 - - 参数类型:int + - 参数类型:Int - 默认值:2
### 2、SQL - **connector** - - 描述:mysql-x + - 描述:连接类型 - 必选:是 - 参数类型:String - - 默认值:无 + - 默认值:db2-x
- **url** - - 描述:jdbc:mysql://localhost:3306/test + - 描述:jdbc:db2://hostname:port/test - 必选:是 - 参数类型:String - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -296,9 +304,10 @@
## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 是否支持 | 类型名称| | --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-sink.md" new file mode 100644 index 0000000000..74ddcbd3e1 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-sink.md" @@ -0,0 +1,181 @@ +# DM Sink + +## 一、介绍 + +DM sink + +## 二、支持版本 + +DM7、DM8 + + +## 三、插件名称 + +| Sync | dmsink、dmwriter | +| ---- |-----------------| +| SQL | | + + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["table"], + "schema":"public" + }] + ``` + +
+ +- **jdbcUrl** + + - 描述:针对关系型数据库的jdbc连接字符串,[达梦官方文档](http://www.dameng.com/down.aspx?TypeId=12&FId=t14:12:14) + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **schema** + + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:用户名 +
+ +- **table** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **fullcolumn** + + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **preSql** + + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **postSql** + + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **writeMode** + + - 描述:控制写入数据到目标表采用 insert into 或者 merge into 语句 + - 必选:是 + - 所有选项:insert/update + - 参数类型:String + - 默认值:insert +
+ +- **batchSize** + + - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 + - 必选:否 + - 参数类型:int + - 默认值:1024 +
+ +- **updateKey** + + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和,应用会以insert的方式写入数据; + - 必选:否 + - 参数类型:Map + - 示例:"updateKey": {"key": ["id"]} + - 默认值:无 +
+ +- **semantic** + + - 描述:sink端是否支持二阶段提交 + - 注意: + - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; + - 当前只支持exactly-once 和at-least-once + - 必选:否 + - 参数类型:String + - 示例:"semantic": "exactly-once" + - 默认值:at-least-once +
+ +- **compatibleMode** + + - 描述:sink端是否支持二阶段提交 + - 注意: + - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; + - 当前只支持exactly-once 和at-least-once + - 必选:否 + - 参数类型:String + - 示例:"semantic": "exactly-once" + - 默认值:at-least-once +
+ + +## 五、数据类型 + + +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | CHAR、CHARACTER、VARCHAR、VARCHAR2、CLOB、TEXT、LONG、LONGVARCHAR、ENUM、SET、JSON、DECIMAL、NUMBERIC、DEC、NUMER、INT、INTEGER、TINYINT、BYTE、BYTES、SMALLINT、BIGINT、BINARY、VARBINARY、BLOB、TINYBLOB、MEDIUMBLOB、LONGBLOB、GEOMETER、IMAGE、REAL、FLOAT、DOUBLE、DOUBLE PRECISION、BIT、YEAR、DATE、TIME、TIMESTAMP、DATETIME | +| 不支持 | PLS_INTEGER、LONGVARBINARY、BFILE、TIME WITH TIME ZONE、TIMESTAMP WITH TIME ZONE、TIME WITH LOCAL TIME ZONE、INTERVAL YEAR、INTERVAL YEAR、INTERVAL MONTH、INTERVAL DAY、INTERVAL HOUR、INTERVAL MINUTE、INTERVAL SECONDE、INTERVAL YEAR TO MONTH、INTERVAL DAY TO HOUR、INTERVAL YEAR TO MINUTE、INTERVAL DAY TO SECONDE、INTERVAL HOUR TO MINUTE、INTERVAL HOUR TO SECOND、INTERVAL MINUTE TO SECONDE、BOOL、BOOLEAN、%TYPE%、%ROWTYPE、记录类型、数组类型、集合类型 | + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-source.md" new file mode 100644 index 0000000000..f7504bb00f --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/dm/dm-source.md" @@ -0,0 +1,230 @@ +# DM Source + +## 一、介绍 + +支持从DM离线读取 + +## 二、支持版本 + +DM7、DM8 + +## 三、插件名称 + +| Sync | dmsource、dmreader | +| ---- |-------------------| +| SQL | | + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["table"], + "schema":"public" + }] + ``` +
+ +- **jdbcUrl** + + - 描述:针对关系型数据库的jdbc连接字符串,[达梦官方文档](http://www.dameng.com/down.aspx?TypeId=12&FId=t14:12:14) + - 必选:是 + - 参数类型:string + - 默认值:用户名 +
+ +- **schema** + + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ +- **table** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **fetchSize** + + - 描述:一次性从数据库中读取多少条数据,DM默认fetchSize大小为10。当fetchSize设置过小时导致频繁读取数据会影响查询速度,以及数据库压力。当fetchSize设置过大时在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否z + - 参数类型:int + - 默认值:1024 +
+ +- **where** + + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > + time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **splitPk** + + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun将报错。 + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **queryTimeOut** + + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ +- **customSql** + + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:需要读取的字段。 + + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ```bash + "column":["*"] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + - 属性说明: + + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + - 必选:是 + + - 默认值:无 +
+ +- **polling** + + - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的起始位置。 + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **pollingInterval** + + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 参数类型:long + - 默认值:5000 +
+ +- **increColumn** + + - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 参数类型:String或int + - 默认值:无 +
+ +- **startLocation** + + - 描述:增量查询起始位置 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **useMaxFunc** + + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **requestAccumulatorInterval** + + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 参数类型:int + - 默认值:2 +
+ +## 五、数据类型 + +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | CHAR、CHARACTER、VARCHAR、VARCHAR2、CLOB、TEXT、LONG、LONGVARCHAR、ENUM、SET、JSON、DECIMAL、NUMBERIC、DEC、NUMER、INT、INTEGER、TINYINT、BYTE、BYTES、SMALLINT、BIGINT、BINARY、VARBINARY、BLOB、TINYBLOB、MEDIUMBLOB、LONGBLOB、GEOMETER、IMAGE、REAL、FLOAT、DOUBLE、DOUBLE PRECISION、BIT、YEAR、DATE、TIME、TIMESTAMP、DATETIME | +| 不支持 | PLS_INTEGER、LONGVARBINARY、BFILE、TIME WITH TIME ZONE、TIMESTAMP WITH TIME ZONE、TIME WITH LOCAL TIME ZONE、INTERVAL YEAR、INTERVAL YEAR、INTERVAL MONTH、INTERVAL DAY、INTERVAL HOUR、INTERVAL MINUTE、INTERVAL SECONDE、INTERVAL YEAR TO MONTH、INTERVAL DAY TO HOUR、INTERVAL YEAR TO MINUTE、INTERVAL DAY TO SECONDE、INTERVAL HOUR TO MINUTE、INTERVAL HOUR TO SECOND、INTERVAL MINUTE TO SECONDE、BOOL、BOOLEAN、%TYPE%、%ROWTYPE、记录类型、数组类型、集合类型 | + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/doris/dorisbatch-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/doris/dorisbatch-sink.md" new file mode 100644 index 0000000000..12ef43dcf2 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/doris/dorisbatch-sink.md" @@ -0,0 +1,127 @@ +# Doris batch Sink + +## 一、介绍 +Doris batch Sink插件支持向Doris数据库写入数据 + +## 二、支持版本 +Doris 0.14.x + +## 三、插件名称 +| Sync | dorisbatchsink、dorisbatchwriter | +| --- | --- | +| SQL | | + +## 四、插件参数 + + +### 1.Sync + +- **feNodes** + - 描述:连接Doris的Fe Nodes 地址 + - 必选:是 + - 字段类型:List + - 示例: +```json +"feNodes": ["127.0.0.1:8030"] +``` + +- 默认值:无 + +
+ + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + + +
+ +- **database** + - 描述:写入Doris的库名 + - 必选:是 + - 字段类型:String + - 默认值:无 + + +
+ +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + + +
+ +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + + +
+ +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 字段类型:List + - 默认值:无 + + +
+ +- **fieldDelimiter** + - 描述:写入Doris数值的字段分隔符 + - 必选:否 + - 字段类型:String + - 默认值:\t + + +
+ +- **lineDelimiter** + - 描述:写入Doris数值的行分隔符 + - 必选:否 + - 字段类型:String + - 默认值:\n + + +
+ +- **loadProperties** + - 描述:针对Doris写入任务的系统参数,主要是针对Doris的特定配置 + - 必选:否 + - 字段类型:Object + - 默认值:无 + + +
+ +- **batchSize** + - 描述:批量写入Doris的数据量大小 + - 必选:否 + - 字段类型:int + - 默认值:1000 + + +
+ + + + + + +## 五、数据类型 + +| 是否支持 | 数据类型 | +| -------- | ------------------------------------------------------------ | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、LARGEINT、FLOAT、DOUBLE、DECIMAL、DATE、DATETIME、CHAR、VARCHAR、STRING | + + +## 六、脚本示例 +见项目内`**chunjun-examples**`文件夹。 diff --git a/docs/connectors/elasticsearch7/es7-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-lookup.md" similarity index 72% rename from docs/connectors/elasticsearch7/es7-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-lookup.md" index e8b645f743..f6a0afa38c 100644 --- a/docs/connectors/elasticsearch7/es7-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-lookup.md" @@ -1,114 +1,139 @@ -# 一、介绍 +# ElasticSearch Lookup + +## 一、介绍 ElasticSearch Lookup插件支持从现有的ElasticSearch集群读取指定index中的数据,并作为维表进行与主表进行关联。目前维表支持全量维表和异步维表。 -# 二、支持版本 +## 二、支持版本 Elasticsearch 7.x -# 三、插件名称 +## 三、插件名称 |类型|名称| | --- | --- | | SQL | elasticsearch7-x | +## 四、参数说明 +### 1、SQL -​
-# 四、参数说明 -## 1、SQL - -- hosts +- **hosts** - 描述:Elasticsearch集群的连接地址。eg: "localhost:9200",多个地址用分号作为分隔符。 - 必选:是 - - 参数类型:List + - 参数类型:List - 默认值:无 -- index + + +- **index** - 描述:指定访问Elasticsearch集群的index名称 - 必选:是 - 参数类型:String - 默认值:无 -- username + + +- **username** - 描述:开启basic认证之后的用户名 - 必须:否 - 参数类型:String - 默认值:无 -- password + + +- **password** - 描述:开启basic认证之后的密码 - 必须:否 - 参数类型:String - 默认值:无 -- client.connect-timeout + + +- **client.connect-timeout** - 描述:ES Client最大连接超时时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- client.socket-timeout + + +- **client.socket-timeout** - 描述:ES Client最大socket超时时间。 - 必须:否 - 参数类型:Integer - 默认值:1800000 -- client.keep-alive-time + + +- **client.keep-alive-time** - 描述:ES Client会话最大保持时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- client.request-timeout + + +- **client.request-timeout** - 描述:ES Client最大请求超时时间。 - 必须:否 - 参数类型:Integer - 默认值:2000 -- client.max-connection-per-route + + +- **client.max-connection-per-route** - 描述:每一个路由值的最大连接数量 - 必须:否 - 参数类型:Integer - 默认值:10 -- lookup.cache-type + + +- **lookup.cache-type** - 描述:维表类型。eg: all 或者 lru - 必须:否 - 参数类型:String - 默认值:LRU -- lookup.cache-period + + +- **lookup.cache-period** - 描述:全量维表周期时间 - 必须:否 - 参数类型:Long - 默认值:3600 * 1000L -- lookup.cache.max-rows + + +- **lookup.cache.max-rows** - 描述:维表缓存的最大条数 - 必须:否 - 参数类型:Long - 默认值:1000L -- lookup.cache.ttl + + +- **lookup.cache.ttl** - 描述:缓存生命周期 - 必须:否 - 参数类型:Long - 默认值:60 * 1000L -- lookup.error-limit + + +- **lookup.error-limit** - 描述:维表数据不合规条数 - 必须:否 - 参数类型:Long - 默认值:Long.MAX_VALUE -- lookup.fetch-size + + +- **lookup.fetch-size** - 描述:抓取维表数据的条数 - 必须:否 - 参数类型:Integer - 默认值:1000L -- lookup.parallelism + + +- **lookup.parallelism8** - 描述:维表并行度 - 必须:否 - 参数类型:Integer - 默认值:1 -# 五、数据类型 -| ​支持 | BOOLEAN | +## 五、数据类型 +|是否支持 | 类型名称 | | --- | --- | -| | INTEGER | -| | DECIMAL | -| | TIMESTAMP | -| | DOUBLE | -| | FLOAT | -| | DATE | -| | VARCHAR | - -# 六、脚本示例 -见项目内ChunJun:Local:Test模块中的demo文件夹。 +| 支持 |INTEGER,FLOAT,DOUBLE,LONG,DATE,TEXT,BYTE,BINARY,OBJECT,NESTED| +| 不支持 | IP,GEO_POINT,GEO_SHAPE| + +## 六、脚本示例 +见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/elasticsearch7/es7-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-sink.md" similarity index 51% rename from docs/connectors/elasticsearch7/es7-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-sink.md" index 643df06627..0969c78da6 100644 --- a/docs/connectors/elasticsearch7/es7-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-sink.md" @@ -1,12 +1,11 @@ -# 一、介绍 +# ElasticSearch Sink +## 一、介绍 ElasticSearch Sink插件支持将数据写入到指定的index中。 -​ -# 二、支持版本 +## 二、支持版本 Elasticsearch 7.x -​ -# 三、插件名称 +## 三、插件名称 | 类型|名称| @@ -14,151 +13,275 @@ Elasticsearch 7.x | Sync | elasticsearch7writer | | SQL | elasticsearch7-x | -​ -# 四、参数说明 +## 四、参数说明 +### 1、数据同步 - -## 1、数据同步 - -- hosts - - 描述:Elasticsearch集群的连接地址。eg: ["localhost:9200"] +- **hosts** + - 描述:Elasticsearch集群的连接地址。eg: "localhost:9200",多个地址用分号作为分隔符。 - 必选:是 - - 参数类型:List + - 参数类型:List - 默认值:无 -- index + + + +- **index** - 描述:指定访问Elasticsearch集群的index名称 - 必选:是 - 参数类型:String - 默认值:无 -- username + + + +- **sername** - 描述:开启basic认证之后的用户名 - 必须:否 - 参数类型:String - 默认值:无 -- password + + + +- **password** - 描述:开启basic认证之后的密码 - 必须:否 - 参数类型:String - 默认值:无 -- batchSize + + + + +- **batchSize** - 描述:批量写入数据的条数 - 必须:否 - 参数类型:Integer - 默认值:1 -- ids - - 描述:用来标识文档Id的字段列表 + + + + +- **ids** + - 描述:指定某些字段生成文档id,若不指定则自动生成 - 必须:否 - - 参数类型:string数组 + - 参数类型:List - 默认值:无 - - 注意:当需要对数据进行更新和删除操作时,必需指定 -- keyDelimiter + + + + +- **keyDelimiter** - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - 必须:否 - 参数类型:无 - 默认值:"_" -- column + + + + +- **column** - 描述:需要读取的字段 - 注意:不支持*格式 - 格式: -- connectTimeout + + ``` + "column": [{ + "name": "col", -- 字段名称 + "type": "text", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 + "value": "value" -- 常量列的值 + }] + ``` + +- **connectTimeout** - 描述:ES Client最大连接超时时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- socketTimeout + + + + +- **socketTimeout** - 描述:ES Client最大socket超时时间。 - 必须:否 - 参数类型:Integer - 默认值:1800000 -- keepAliveTime + + + + +- **keepAliveTime** - 描述:ES Client会话最大保持时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- requestTimeout + + + + +- **requestTimeout** - 描述:ES Client最大请求超时时间。 - 必须:否 - 参数类型:Integer - 默认值:2000 -- maxConnPerRoute + + + + +- **maxConnPerRoute** - 描述:每一个路由值的最大连接数量 - 必须:否 - 参数类型:Integer - 默认值:10 -``` -"column": [{ - "name": "col", -- 字段名称,可使用多级格式查找 - "type": "string", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 - "value": "value" -- 常量列的值 -}] -``` -​ + + + +- **sslConfig** + - 描述:开启ssl连接认证需要的配置项 + - useLocalFile:是否使用本地文件 + - fileName:文件名,使用本地文件时,文件路径为:filePath/fileName,使用sftp时,文件路径为:path/fileName + - filePath:文件所在上级目录 + - keyStorePass:使用证书文件的密码,在生成证书文件时所指定的密码,若无则无需配置 + - type:证书类型,目前支持ca(ca.crt)和pkcs12(xxx.p12)两种类型的证书文件,可选值:ca/pkcs12 + - sftpConf:sftp配置 + - 必须:否 + - 参数类型:Map + - 示例: + ``` + "sslConfig": { + "useLocalFile":false, + "fileName":"ca.crt", + "filePath":"/Users/edy/Downloads", + "keyStorePass":"", + "type":"ca", + "sftpConf": { + "path":"/data/sftp/ssl", + "password":"dtstack", + "port":"22", + "auth":"1", + "host":"127.0.0.1", + "username":"root" + } + } + ``` ## 2、SQL -- hosts +- **hosts** - 描述:Elasticsearch集群的连接地址。eg: "localhost:9200", 多地址用分号分隔符隔开。 - 必选:是 - - 参数类型:List + - 参数类型:List - 默认值:无 -- index + + + +- **index** - 描述:指定访问Elasticsearch集群的index名称 - 必选:是 - 参数类型:String - 默认值:无 -- username + + + +- **username** - 描述:开启basic认证之后的用户名 - 必须:否 - 参数类型:String - 默认值:无 -- password + + + +- **password** - 描述:开启basic认证之后的密码 - 必须:否 - 参数类型:String - 默认值:无 -- sink.bulk-flush.max-actions + + + +- **sink.bulk-flush.max-actions** - 描述:批量写入es数据的条数 - 必须:否 - 参数类型:Integer - 默认值:1 -- document-id.key-delimiter + + + +- **document-id.key-delimiter** - 描述:文档id之间的分隔符号,eg:“${col1}_${col2}” - 必须:否 - 参数类型:String - 默认值:"_" -- client.connect-timeout + + + +- **client.connect-timeout** - 描述:ES Client最大连接超时时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- client.socket-timeout + + + +- **client.socket-timeout** - 描述:ES Client最大socket超时时间。 - 必须:否 - 参数类型:Integer - 默认值:1800000 -- client.keep-alive-time + + + +- **client.keep-alive-time** - 描述:ES Client会话最大保持时间。 - 必须:否 - 参数类型:Integer - 默认值:5000 -- client.request-timeout + + + +- **client.request-timeout** - 描述:ES Client最大请求超时时间。 - 必须:否 - 参数类型:Integer - 默认值:2000 -- client.max-connection-per-route + + + +- **client.max-connection-per-route** - 描述:每一个路由值的最大连接数量 - 必须:否 - 参数类型:Integer - 默认值:10 + + + +- **security.ssl-keystore-file** + - 描述:ssl keystore认证文件名 + - 必须: 否 + - 参数类型:String + - 默认值:无 + + + +- **security.ssl-keystore-password** + - 描述:ssl keystore认证文件密码,如果存在的话 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **security.ssl-type** + - 描述:证书类型,目前支持ca(ca.crt)和pkcs12(xxx.p12)两种类型的证书文件,可选值:ca/pkcs12 + - 必须:否 + - 参数类型:String + - 默认值:无 # 五、数据类型 |是否支持 | 类型名称 | | --- | --- | -| 支持 |INTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE| -| 不支持 | IP,binary, nested, object| +| 支持 |INTEGER,FLOAT,DOUBLE,LONG,DATE,TEXT,BYTE,BINARY,OBJECT,NESTED| +| 不支持 | IP,GEO_POINT,GEO_SHAPE| # 六、脚本示例 -见项目内`ChunJun:Local:Test`模块中的`demo文件夹。 +见项目内`chunjun-examples`文件夹。 + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-source.md" new file mode 100644 index 0000000000..af257bc1a8 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/elasticsearch/es7-source.md" @@ -0,0 +1,259 @@ +# ElasticSearch Source + +## 一、介绍 +ElasticSearch Source插件支持从现有的ElasticSearch集群读取指定index中的数据。 + +## 二、支持版本 +Elasticsearch 7.x + +## 三、插件名称 + +| 类型|名称| +| --- | --- | +| Sync | elasticsearch7reader | +| SQL | elasticsearch7-x | + + +## 四、参数说明 +### 1、数据同步 + +- **hosts** + - 描述:Elasticsearch集群的连接地址。 eg: "localhost:9200",多个地址用分号作为分隔符。 + - 必选:是 + - 参数类型:List + - 默认值:无 + + + +- **index** + - 描述:指定访问Elasticsearch集群的index名称 + - 必选:是 + - 参数类型:String + - 默认值:无 + + + +- **username** + - 描述:开启basic认证之后的用户名 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **password** + - 描述:开启basic认证之后的密码 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **batchSize** + - 描述:批量读取数据的条数 + - 必须:否 + - 参数类型:Integer + - 默认值:1 + + + +- **column** + - 描述:需要读取的字段 + - 注意:不支持*格式 + - 参数类型:List + - 格式: + + ``` + "column": [{ + "name": "col", -- 字段名称 + "type": "text", -- 字段类型,当name没有指定时,则返回常量列,值为value指定 + "value": "value" -- 常量列的值 + }] + ``` + - 默认值:无 + + + +- **connectTimeout** + - 描述:ES Client最大连接超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:5000 + + + +- **socketTimeout** + - 描述:ES Client最大socket超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:1800000 + + + +- **keepAliveTime** + - 描述:ES Client会话最大保持时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:5000 + + + +- **requestTimeout** + - 描述:ES Client最大请求超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:2000 + + + +- **maxConnPerRoute** + - 描述:每一个路由值的最大连接数量 + - 必须:否 + - 参数类型:Integer + - 默认值:10 + + + +- **sslConfig** + - 描述:开启ssl连接认证需要的配置项 + - useLocalFile:是否使用本地文件 + - fileName:文件名,使用本地文件时,文件路径为:filePath/fileName,使用sftp时,文件路径为:path/fileName + - filePath:文件所在上级目录 + - keyStorePass:使用证书文件的密码,在生成证书文件时所指定的密码,若无则无需配置 + - type:证书类型,目前支持ca(ca.crt)和pkcs12(xxx.p12)两种类型的证书文件,可选值:ca/pkcs12 + - sftpConf:sftp配置 + - 必须:否 + - 参数类型:Map + - 示例: + ``` + "sslConfig": { + "useLocalFile":false, + "fileName":"ca.crt", + "filePath":"/Users/edy/Downloads", + "keyStorePass":"", + "type":"ca", + "sftpConf": { + "path":"/data/sftp/ssl", + "password":"dtstack", + "port":"22", + "auth":"1", + "host":"127.0.0.1", + "username":"root" + } + } + ``` + + +#### 2、SQL + +- **hosts** + - 描述:Elasticsearch集群的连接地址。eg: "localhost:9200",多个地址用分号作为分隔符。 + - 必选:是 + - 参数类型:List + - 默认值:无 + + + +- **index** + - 描述:指定访问Elasticsearch集群的index名称 + - 必选:是 + - 参数类型:String + - 默认值:无 + + + +- **username** + - 描述:开启basic认证之后的用户名 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **password** + - 描述:开启basic认证之后的密码 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **sink.bulk-flush.max-actions** + - 描述:一次性读取es数据的条数 + - 必须:否 + - 参数类型:Integer + - 默认值:1 + + + +- **client.connect-timeout** + - 描述:ES Client最大连接超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:5000 + + + +- **client.socket-timeout** + - 描述:ES Client最大socket超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:1800000 + + + +- **client.keep-alive-time** + - 描述:ES Client会话最大保持时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:5000 + + + +- **client.request-timeout** + - 描述:ES Client最大请求超时时间。 + - 必须:否 + - 参数类型:Integer + - 默认值:2000 + + + +- **client.max-connection-per-route** + - 描述:每一个路由值的最大连接数量 + - 必须:否 + - 参数类型:Integer + - 默认值:10 + + + +- **security.ssl-keystore-file** + - 描述:ssl keystore认证文件名 + - 必须: 否 + - 参数类型:String + - 默认值:无 + + + +- **security.ssl-keystore-password** + - 描述:ssl keystore认证文件密码,如果存在的话 + - 必须:否 + - 参数类型:String + - 默认值:无 + + + +- **security.ssl-type** + - 描述:证书类型,目前支持ca(ca.crt)和pkcs12(xxx.p12)两种类型的证书文件,可选值:ca/pkcs12 + - 必须:否 + - 参数类型:String + - 默认值:无 + +## 五、数据类型 + +|是否支持 | 类型名称 | +| --- | --- | +| 支持 |INTEGER,FLOAT,DOUBLE,LONG,DATE,TEXT,BYTE,BINARY,OBJECT,NESTED| +| 不支持 | IP,GEO_POINT,GEO_SHAPE| + +## 六、脚本示例 +见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/gbase/gbase-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-lookup.md" similarity index 96% rename from docs/connectors/gbase/gbase-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-lookup.md" index 5579679634..3ee32d1769 100644 --- a/docs/connectors/gbase/gbase-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-lookup.md" @@ -22,7 +22,7 @@ GBase8a(8.6.2.43)
- **url** - - 描述:jdbc:gbase://localhost:9042/test,GBase jdbc 链接 + - 描述:jdbc:gbase://localhost:9042/test - 必选:是 - 参数类型:String - 默认值:无 @@ -99,8 +99,9 @@ GBase8a(8.6.2.43)
## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 是否支持 | 数据类型 | | --- | --- | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | | 暂不支持 | ARRAY、MAP、STRUCT、UNION | diff --git a/docs/connectors/gbase/gbase-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-sink.md" similarity index 95% rename from docs/connectors/gbase/gbase-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-sink.md" index 783d431edb..ac273949e7 100644 --- a/docs/connectors/gbase/gbase-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-sink.md" @@ -20,7 +20,7 @@ GBase8a(8.6.2.43) - 必选:是 - 参数类型:List - 默认值:无 - ```text + ```json "connection": [{ "jdbcUrl": ["jdbc:gbase://0.0.0.1:9042/database?useSSL=false"], "table": ["table"], @@ -37,11 +37,11 @@ GBase8a(8.6.2.43)
- **schema** - - 描述:数据库schema名 + - 描述:GBase 数据库 - 必选:否 - - 参数类型:string + - 参数类型:String - 默认值:无 -
+
- **table** - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 @@ -144,6 +144,13 @@ GBase8a(8.6.2.43) - 默认值:无
+- **schema** + - 描述:GBase 数据库 + - 必选:否 + - 参数类型:String + - 默认值: +
+ - **table-name** - 描述:表名 - 必选:是 @@ -205,9 +212,10 @@ GBase8a(8.6.2.43)
## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 是否支持 | 数据类型 | | --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git a/docs/connectors/gbase/gbase-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-source.md" similarity index 94% rename from docs/connectors/gbase/gbase-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-source.md" index 995ff236f5..cc734ed2c1 100644 --- a/docs/connectors/gbase/gbase-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/gbase/gbase-source.md" @@ -20,7 +20,7 @@ GBase8a(8.6.2.43) - 必选:是 - 参数类型:List - 默认值:无 - ```text + ```json "connection": [{ "jdbcUrl": ["jdbc:gbase://0.0.0.1:9042/database?useSSL=false"], "table": ["table"], @@ -65,8 +65,8 @@ GBase8a(8.6.2.43)
- **fetchSize** - - 描述:一次性从数据库中读取多少条数据,GBase默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2、连接参数useCursorFetch=true。 -注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 描述:一次性从数据库中读取多少条数据,GBase默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0。连接参数useCursorFetch=true。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 - 参数类型:int - 默认值:1024 @@ -115,14 +115,17 @@ GBase8a(8.6.2.43) - 描述:需要读取的字段。 - 格式:支持3种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash + - + ```json "column":["*"] ``` 2.只指定字段名称: - ``` + + ```json "column":["id","name"] ``` 3.指定具体信息: + ```json "column": [{ "name": "col", @@ -197,6 +200,13 @@ GBase8a(8.6.2.43) - 默认值:无
+- **schema** + - 描述:GBase 数据库 + - 必选:否 + - 参数类型:String + - 默认值: +
+ - **table-name** - 描述:表名 - 必选:是 @@ -226,7 +236,7 @@ GBase8a(8.6.2.43)
- **scan.parallelism** - - 描述:并行度,间隔轮训目前不支持多并行度 + - 描述:并行度,间隔轮询目前不支持多并行度 - 必选:否 - 参数类型:String - 默认值:无 @@ -261,7 +271,7 @@ GBase8a(8.6.2.43)
- **scan.increment.column** - - 描述:增量字段名称,如果配置了该字段,目前并行度只能为1。非必填,无默认 + - 描述:增量字段名称,如果配置了该字段,目前并行度只能为1。非必填,无默认。 - 必选:否 - 参数类型:String - 默认值:无 @@ -296,9 +306,10 @@ GBase8a(8.6.2.43)
## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 是否支持 | 数据类型 | | --- | --- | -| 暂不支持 | ARRAY、MAP、STRUCT、UNION | +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git a/docs/connectors/greenplum/greenplum-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-sink.md" similarity index 95% rename from docs/connectors/greenplum/greenplum-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-sink.md" index 24ae07cddd..3953aab4b9 100644 --- a/docs/connectors/greenplum/greenplum-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-sink.md" @@ -8,8 +8,9 @@ Greenplum 5及以上 ## 三、插件名称 -| Sync | greenplumsink、greenplumwriter | +|Type | Name | | --- | --- | +| Sync | greenplumsink、greenplumwriter | | SQL | greenplum-x | @@ -20,13 +21,14 @@ Greenplum 5及以上 - 必选:是 - 参数类型:List - 默认值:无 - ```text + + ```json "connection": [{ - "jdbcUrl": ["jdbc:pivotal:greenplum://localhost:5432"], + "jdbcUrl": ["jdbc:pivotal:greenplum://localhost:5432;DatabaseName=postgres"], "table": ["table"], "schema":"public" }] - ``` + ```
- **jdbcUrl** @@ -144,6 +146,13 @@ Greenplum 5及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -205,9 +214,10 @@ Greenplum 5及以上
## 五、数据类型 -| 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | +|是否支持 | 类型名称| | --- | --- | -| 暂不支持 | ARRAY等 | +| 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | +| 不支持 | ARRAY等 | ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/greenplum/greenplum-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-source.md" similarity index 96% rename from docs/connectors/greenplum/greenplum-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-source.md" index c5c696f379..22fd950475 100644 --- a/docs/connectors/greenplum/greenplum-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/greenplum/greenplum-source.md" @@ -19,13 +19,14 @@ Greenplum 5及以上 - 必选:是 - 参数类型:List - 默认值:无 - ```text + + ```json "connection": [{ - "jdbcUrl": ["jdbc:pivotal:greenplum://localhost:5432"], + "jdbcUrl": ["jdbc:pivotal:greenplum://localhost:5432;DatabaseName=postgres"], "table": ["table"], "schema":"public" }] - ``` + ```
- **jdbcUrl** @@ -114,14 +115,17 @@ Greenplum 5及以上 - 描述:需要读取的字段。 - 格式:支持3种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash + + ```json "column":["*"] ``` 2.只指定字段名称: - ``` + + ```json "column":["id","name"] ``` 3.指定具体信息: + ```json "column": [{ "name": "col", @@ -196,6 +200,13 @@ Greenplum 5及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -294,9 +305,10 @@ Greenplum 5及以上
## 五、数据类型 -| 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | +|是否支持 | 类型名称| | --- | --- | -| 暂不支持 | ARRAY等 | +| 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | +| 不支持 | ARRAY等 | ## 六、脚本示例 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-lookup.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-lookup.md" new file mode 100644 index 0000000000..d5a64768f4 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-lookup.md" @@ -0,0 +1,85 @@ +## 一、介绍 +Hbase Lookup查询数据。 + +## 二、支持版本 +hbase1.4 + +## 三、插件名称 +| SQL | hbase14-x | +| --- |---------| + + + +## 四、参数说明 + +- **connector** + - 描述:hbase14-x + - 必选:是 + - 默认值:无 + + +- **properties.zookeeper.quorum** + - 描述:zk地址 + - 必选:是 + - 示例:xxx:2181,xxx:2181,xxx:2181 + - 默认值:无 + + +- **properties.zookeeper.znode.parent** + - 描述:hbase在zk目录 + - 必选:否 + - 默认值:/hbase + + +- **null-string-literal** + - 描述:空值字符串代替 + - 必选:否 + - 默认值:"null" + + +- **lookup.cache-type** + - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU + - 必选:否 + - 默认值:LRU + + + +- **lookup.error-limit** + - 描述:异步维表加载数据错误数量上限 + - 必选:LRU异步 + - 默认值:Long.MAX_VALUE + + +- **lookup.cache-period** + - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) + - 必选:否 + - 默认值:3600000 + + +- **lookup.async-timeout** + - 描述:维表查询超时时间 + - 必选:否 + - 默认值:10000 + + +- **lookup.cache.max-rows** + - 描述:lru维表缓存数据的条数 + - 必选:否 + - 默认值:10000 + + +- **lookup.cache.ttl** + - 描述:lru维表缓存数据的时间 + - 必选:否 + - 默认值:60000 + + +- **lookup.parallelism** + - 描述:维表并行度 + - 必选:否 + - 默认值:无 +## +## 五、脚本示例 +见项目内`chunjun-examples`文件夹。 + + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-sink.md" new file mode 100644 index 0000000000..06aec4d0c7 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-sink.md" @@ -0,0 +1,171 @@ +# Hbase Sink + +## 一、介绍 + +hbase sink + +## 二、支持版本 + +hbase1.4 + + +## 三、插件名称 + +| Sync | hbasesink、hbasewriter | +| ---- |-----------------------| +| SQL | hbase14-x | + + +## 四、参数说明 + +### 1、Sync + +- **table** + - 描述:表名 + - 必选:是 + - 类型:String + - 默认值:无 + + +- **encoding** + - 描述:编码 + - 必选:否 + - 类型:string + - 默认值:utf-8 + + +- **nullMode** + - 描述:字段值为空时写入模式 + - 必选:否 + - 可选:SKIP:跳过,此字段不写入,EMPTY:空字节数组代替 + - 参数类型:string + - 默认值:SKIP + + + +- **walFlag** + - 描述:是否跳过WAL + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **writeBufferSize** + - 描述:设置HBae client的写buffer大小,单位字节 + - 必选:否 + - 参数类型:Long + - 默认值: 8 * 1024 * 1024 +
+ + + +- **rowkeyExpress** + - 描述: 用于构造rowkey的描述信息,采用字符串格式,形式如下 + 字符串格式为:$(cf:col),可以多个字段组合:$(cf:col1)_$(cf:col2), + 可以使用md5函数:md5($(cf:col)) + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ + + +- **versionColumnIndex** + - 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。 + 注意,在hbase中查询默认会显示时间戳最大的数据,因此简单查询可能会出现看不到更新的情况,需要加过滤条件查询。 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **versionColumnValue** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ + +### 2、SQL + +- **connector** + - 描述:hbase14-x + - 必选:是 + - 参数类型:String + - 默认值:无 + + +- **properties.zookeeper.znode.parent** + - 描述:hbase在zk的路径 + - 必选:否 + - 参数类型:string + - 默认值:/hbase +
+ + +- **properties.zookeeper.quorum** + - 描述:zk地址 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ + + +- **table-name** + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ + + +- **sink.buffer-flush.max-size** + - 描述:每个写请求缓冲行的最大内存大小。这样可以提高HBase写数据的性能,但可能会增加时延。可以设置为'0'来禁用它。 + - 必选:否 + - 参数类型:String + - 默认值:2mb +
+ + +- **sink.buffer-flush.max-rows** + - 描述:每个写入请求要缓冲的最大行数。这样可以提高HBase写数据的性能,但可能会增加时延。可以设置为'0'来禁用它。 + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ + + +- **sink.buffer-flush.interval** + - 描述:批量写时间间隔,单位:毫秒 + - 必选:否 + - 参数类型:String + - 默认值:10000 +
+ + + +- **sink.parallelism** + - 描述:写入结果的并行度 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ + + +## 五、数据类型 + + +| 是否支持 | 类型名称 | +| :--------------: |:------------------------------------------:| +| 支持 | INT、LONG、DOUBLE、FLOAT、SHORT、BOOLEAN、STRING | +| 暂不支持 | | +| 仅在 Sync 中支持 | | + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-source.md" new file mode 100644 index 0000000000..b2622ddab5 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hbase/hbase-source.md" @@ -0,0 +1,161 @@ +# Hbase Source +## 1. Introduce +Hbase Source + + +## 2. Version Support +hbase1.4 + + +## 3. Connector Name +| Sync | hbasesource、hbasereader | +| --- |-------------------------| +| SQL | hbase14-x | + + + +## 4. Parameter description +### 1、SYNC + + +- **table** + - 描述:表名 + - 必选:是 + - 类型:String + - 默认值:无 + + +- **startRowkey** + - 描述:rowKey起始点 + - 必选:否 + - 类型:String + - 默认值:无 + + + +- **endRowkey** + - 描述:rowKey结束点 + - 必选:否 + - 类型:String + - 默认值:无 + + + +- **isBinaryRowkey** + - 描述:rowkey是否是BytesBinary + - 必选:否 + - 类型:Boolean + - 默认值:false + + +- **scanCacheSize** + - 描述:客户端rpc每次fetch最大行数 + - 必选:否 + - 类型:Long + - 默认值:1000 + + + +- **encoding** + - 描述:编码 + - 必选:否 + - 类型:string + - 默认值:utf-8 + + + +- **hbaseConfig** + - 描述:hbase-site里的相关配置 以及 kerberos相关配置 + - 必选:是 + - 类型:Map + - 默认值:无 + + + +- **column** + - 描述:需要读取的列族。 + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - 必选:是 + - 字段类型:List + - 默认值:无 + +### 2、SQL + +- **connector** + - 描述:hbase14-x + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ + +- **table-name** + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ + +- **properties.zookeeper.znode.parent** + - 描述:hbase在zk的路径 + - 必选:否 + - 参数类型:string + - 默认值:/hbase +
+ + +- **properties.zookeeper.quorum** + - 描述:zk地址 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ + +- **null-string-literal** + - 描述:空值字符串代替 + - 必选:否 + - 默认值:"null" +
+ + + +- **security.kerberos.principal** + - 描述:kerberos的principal + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ + +- **security.kerberos.keytab** + - 描述:kerberos的keytab文件路径 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ + + +- **security.kerberos.krb5conf** + - 描述:kdc的krb5conf配置文件 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ + +## 5. Data Type + +| 是否支持 | 类型名称 | +| :--------------: |:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------:| +| 支持 | BOOLEAN、TINYINT、INT8、UINT8、SMALLINT、UINT16、INT16、INTEGER、INTERVALYEAR、INTERVALQUARTER、INTERVALMONTH、INTERVALWEEK、INTERVALDAY、INTERVALHOUR、INTERVALMINUTE INTERVALSECOND、INT32、INT、UINT32、 UINT64、 INT64、 BIGINT、 FLOAT、FLOAT32、 DECIMAL、 DECIMAL32、DECIMAL64、DECIMAL128、 DEC、DOUBLE、FLOAT64、UUID、COLLECTION、BLOB、LONGTEXT、TINYTEXT、TEXT、CHAR、MEDIUMTEXT、TINYBLOB、MEDIUMBLOB、LONGBLOB、BINARY、STRUCT、VARCHAR、STRING、ENUM8、ENUM16、FIXEDSTRING、NESTED、DATE、TIME、TIMESTAMP、DATETIME、NOTHING、NULLABLE、NULL | +| 暂不支持 | | +| 仅在 Sync 中支持 | | + + +## 6. Example +The details are in chunjun-examples dir. diff --git a/docs/connectors/hdfs/hdfs-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hdfs/hdfs-sink.md" similarity index 100% rename from docs/connectors/hdfs/hdfs-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hdfs/hdfs-sink.md" diff --git a/docs/connectors/hdfs/hdfs-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hdfs/hdfs-source.md" similarity index 100% rename from docs/connectors/hdfs/hdfs-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hdfs/hdfs-source.md" diff --git a/docs/connectors/hive/hive-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hive/hive-lookup.md" similarity index 100% rename from docs/connectors/hive/hive-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hive/hive-lookup.md" diff --git a/docs/connectors/hive/hive-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hive/hive-sink.md" similarity index 100% rename from docs/connectors/hive/hive-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/hive/hive-sink.md" diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-sink.md" new file mode 100644 index 0000000000..0e90a7646b --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-sink.md" @@ -0,0 +1,162 @@ +# InfluxDB Sink + +## 一、介绍 + +influxdb sink,不支持断点续传,只能根据time排序,而time非主键不唯一,如果你批量写入数据点,并且所有的写入点没有显示的 timestamp,那么它们将被以相同的 timestamp 写入。 + +## 二、支持版本 + +influxDB 1.x + + +## 三、插件名称 + +| Sync | influxdbsink、influxdbwriter | +| ---- |-----------------------------| +| SQL | | + + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、database、measurement等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "url": ["http://127.0.0.1:8086"], + "measurement": ["measurement"], + "database":"public" + }] + ``` + +
+ +- **url** + + - 描述:连接influxDB的url + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **database** + + - 描述:数据库名 + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **measurement** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **rp** + + - 描述:influxdb中的数据保留多长时间、数据保留几个副本(开源版只能保留一个)、每个shard保留多长时间的数据,创建一个retentionPolicy的要素: + - DURATION:这个描述了保留策略要保留多久的数据。这个机制对于时序型的数据来讲,是非常有用的。 + - SHARD:这个是实际存储influxdb数据的单元。每个shard保留一个时间片的数据,默认是7天。如果你保存1年的数据,那么influxdb会把连续7天的数据放到一个shard中,使用好多个shard来保存数据。 + - SHARD DURATION:这个描述了每个shard存放多数据的时间片是多大。默认7天。需要注意的是,当数据超出了保留策略后,influxdb并不是按照数据点的时间一点一点删除的,而是会删除整个shard group。 + - SHARD GROUP:顾名思义,这个一个shard group包含多个shard。对于开源版的influxdb,这个其实没有什么区别,可以简单理解为一个shard group只包含一个shard,但对于企业版的多节点集群模式来讲,一个shard group可以包含不同节点上的不同shard,这使得influxdb可以保存更多的数据。 + - SHARD REPLICATION:这个描述了每个shard有几个副本。对于开源版来讲,只支持单副本,对于企业版来讲,每个shard可以冗余存储,这样可以避免单点故障。 + - 必选:否 + - 参数类型:String + - 默认值:influxdb默认retention policy +
+ +- **writeMode** + + - 描述:由于influxdb优先考虑create和read数据的性能而不是update和delete,对现有数据的更新是罕见的事件,持续地更新永远不会发生。时间序列数据主要是永远不更新的新数据。因此我们只支持insert操作 + - 必选:否 + - 参数类型:String + - 默认值:insert +
+ +- **enableBatch** + + - 描述:是否开启batch写入 + - 必选:否 + - 参数类型:boolean + - 默认值:true +
+ +- **bufferLimit** + + - 描述:批次,InfluxData建议每个buffer的大小在5000~10000个数据点 + - 必选:否 + - 参数类型:int + - 默认值:10000 +
+ +- **flushDuration** + + - 描述:写入间隔,开启批次写入是从buffer flush进磁盘的间隔(毫秒) + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ +- **flushDuration** + + - 描述:与timestamp字段搭配使用,设置所提供的Unix时间的精度。如果您不指定精度,TSDB For InfluxDB®假设时间戳的精度为纳秒。 + - 必选:否 + - 可选值:ns,u,ms,s,m,h + - 参数类型:string + - 默认值:ns +
+ +- **tag** + + - 描述:哪些字段是tag(measurement的tag对应的key) + - 必选:否 + - 参数类型:list + - 默认值:无 + + ```text + "tags":["id","no"], + ``` + +
+ + +## 五、数据类型 + + +| 是否支持 | 类型名称 | +|:---------------:| :----------------------------------------------------------: | +| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-source.md" new file mode 100644 index 0000000000..ad9b43be36 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/influxdb/influxdb-source.md" @@ -0,0 +1,210 @@ +# InfluxDB Source + +## 一、介绍 + +支持从influxDB离线读取,不支持断点续传(只能根据 time 排序,而 time 非主键,不唯一) + +## 二、支持版本 + +influxDB 1.x + +## 三、插件名称 + +| Sync | influxdbsource、influxdbreader | +| ---- |-------------------------------| +| SQL | | + + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、database、measurement等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "url": ["http://127.0.0.1:8086"], + "measurement": ["measurement"], + "database":"public" + }] + ``` + +
+ +- **url** + + - 描述:连接influxDB的url + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **database** + + - 描述:数据库名 + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **measurement** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **format** + - 描述:响应格式 + - 必选:否 + - 参数类型:string + - 默认值:MSGPACK + - 可选值:MSGPACK/JSON + - 区别:
+ ⅰ. JSON 无法区分浮点数和整数
+ ⅱ. JSON 不支持大于 2^53 的整数
+ ⅲ. JSON 具有有限的性能特征
+ ⅳ. 当 format 为 json 时,time 字段会在反序列化中 double 类型强转 long + 时丢失精度,详情见 https://github.com/influxdata/influxdb-java/issues/517 +
+ +- **fetchSize** + + - 描述:一次性从数据库中读取多少条数据。当fetchSize设置过小时导致频繁读取数据会影响查询速度,以及数据库压力。当fetchSize设置过大时在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ +- **where** + + - 描述:描述:筛选条件,reader插件根据指定的column、table、where条件拼接 InfluxQL,并根据这个 InfluxQL 进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为time > currenttime + - 注意:需符合 InfluxQL 语法,不要添加 order by + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **splitPk** + + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接 InfluxQL ,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 不支持 tags 为切分主键,因为 tags 的类型只能为字符串 + - 不支持 time 为切分主键,因为 time 字段无法参与数学计算 + - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **epoch** + + - 描述:返回的time精度 + - 注意:当 format 为 json 时,time 字段会在反序列化中 double 类型强转 long 时丢失精度,详情见 https://github.com/influxdata/influxdb-java/issues/517 + - 可选值:h、m、s、ms、u、n + - 必选:否 + - 参数类型:String + - 默认值:n +
+ +- **queryTimeOut** + + - 描述:查询超时时间,单位秒。 + - 必选:否 + - 参数类型:int + - 默认值:3 +
+ +- **customSql** + + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 禁止将time字段以外的字段作为order by字段 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:需要读取的字段。 + + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ```bash + "column":["*"] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + - 属性说明: + + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + - 必选:是 + + - 默认值:无 +
+ + + +## 五、数据类型 + +| 是否支持 | 类型名称 | +|:---------------:| :----------------------------------------------------------: | +| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-sink.md" new file mode 100644 index 0000000000..6f4a9168ca --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-sink.md" @@ -0,0 +1,206 @@ + + +## 一、介绍 + +kafka sink + +## 二、支持版本 + +kafka主流版本 + +## 三、插件名称 +| Sync | kafkasink、kafkawriter | +| --- | --- | +| SQL | kafka-x | +| SQL(upsert) | upsert-kafka-x | + + +## 四、参数说明 + +### 1、Sync + +- **topic** + - 描述:消息发送至kafka的topic名称,不支持多个topic + - 必选:是 + - 字段类型:String + - 默认值:无 +- **consumerSettings** + - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 + - 必选:是 + - 字段类型:Map + - 默认值:无 + - 如: +```json +{ + "consumerSettings":{ + "bootstrap.servers":"host1:9092,host2:9092,host3:9092" + } +} +``` + + +- **tableFields** + - 描述:字段映射配置。从reader插件传递到writer插件的的数据只包含其value属性,配置该参数后可将其还原成键值对类型json字符串输出。 + - 注意: + - 若配置该属性,则该配置中的字段个数必须不少于reader插件中读取的字段个数,否则该配置失效; + - 映射关系按该配置中字段的先后顺序依次匹配; + - 必选:否 + - 字段类型:String[] + - 默认值:无 + +### 2、SQL + +具体可以参考:[kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) + +- **connector** + - 描述:kafka-x + - 必选:是 + - 字段类型:String + - 默认值:无 +- **topic** + - 描述:当表用作源时要从中读取数据的主题名称。它还通过用分号分隔主题来支持源的主题列表,如'topic-1;topic-2'. 请注意,只能为源指定“topic-pattern”和“topic”之一。当表用作接收器时,主题名称是要写入数据的主题。接收器不支持注意主题列表。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **topic-pattern** + - 描述:要从中读取的主题名称模式的正则表达式。当作业开始运行时,消费者将订阅名称与指定正则表达式匹配的所有主题。请注意,只能为源指定“topic-pattern”和“topic”之一。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **properties.bootstrap.servers** + - 描述:逗号分隔的 Kafka 代理列表。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **properties.group.id** + - 描述:Kafka source的消费组id,Kafka sink可选。 + - 必选:required by source + - 字段类型:String + - 默认值:无 +- **properties.*** + - 描述:这可以设置和传递任意 Kafka 配置。后缀名称必须与[Kafka 配置文档中](https://kafka.apache.org/documentation/#configuration)定义的配置键匹配。Flink 将删除“属性”。键前缀并将转换后的键和值传递给底层 KafkaClient。例如,您可以通过 禁用自动主题创建'properties.allow.auto.create.topics' = 'false'。但是有一些配置是不支持设置的,因为 Flink 会覆盖它们。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **format** + - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:此选项或'value.format'选项都是必需的。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **key.format** + - 描述:用用于反序列化和序列化 Kafka 消息关键部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:如果定义了密钥格式,则该'key.fields' 选项也是必需的。否则 Kafka 记录将有一个空键。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **key.fields** + - 描述:定义表架构中物理列的显式列表,用于配置键格式的数据类型。默认情况下,此列表为空,因此未定义键。该列表应如下所示'field1;field2'。 + - 必选:否 + - 字段类型:List + - 默认值:无 +- **key.fields-prefix** + - 描述:为键格式的所有字段定义自定义前缀,以避免与值格式的字段发生名称冲突。默认情况下,前缀为空。如果定义了自定义前缀,则表架构 和'key.fields'都将使用前缀名称。在构造密钥格式的数据类型时,将删除前缀,并在密钥格式中使用非前缀名称。请注意,此选项要求'value.fields-include' 必须设置为'EXCEPT_KEY'。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **value.format** + - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多格式选项,请参阅格式页面。注意:此选项或'format'[选项](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)都是必需的。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **value.fields-include** + - 描述:定义如何处理值格式的数据类型中的键列的策略。默认情况下,'ALL'表模式的物理列将包含在值格式中,这意味着键列出现在键和值格式的数据类型中 + - 必选:否 + - 字段类型:枚举 + - 可选的值:[ALL, EXCEPT_KEY] + - 默认值:ALL +- **scan.startup.mode** + - 描述:卡夫卡消费的启动模式,有效值为'earliest-offset','latest-offset','group-offsets','timestamp'和'specific-offsets'。有关更多详细信息,请参阅以下[开始阅读位置](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#start-reading-position)。 + - 必选:否 + - 字段类型:String + - 默认值:group-offsets +- **scan.startup.specific-offsets** + - 描述:在'specific-offsets'启动模式下为每个分区指定偏移量,例如'partition:0,offset:42;partition:1,offset:300'. + - 必选:否 + - 字段类型:String + - 默认值:无 +- **scan.startup.timestamp-millis** + - 描述:从'timestamp'启动模式下使用的指定纪元时间戳(毫秒)开始。 + - 必选:否 + - 字段类型:Long + - 默认值:无 +- **scan.topic-partition-discovery.interval** + - 描述:消费者定期发现动态创建的 Kafka 主题和分区的时间间隔。 + - 必选:否 + - 字段类型:Duration + - 默认值:无 +- **sink.partitioner** + - 描述: 从 Flink 的分区到 Kafka 的分区的输出分区。有效值为 + - default: 使用 kafka 默认分区器对记录进行分区。 + - fixed:每个 Flink 分区最终最多包含一个 Kafka 分区。 + - round-robin:一个 Flink 分区被分发到 Kafka 分区粘性循环。它仅在未指定记录的键时有效。 + - 自定义FlinkKafkaPartitioner子类:例如'org.mycompany.MyPartitioner'. + - 有关更多详细信息,请参阅以下[接收器分区](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#sink-partitioning)。 + - 必选:否 + - 字段类型:String + - 默认值:default +- **sink.semantic** + - 描述:定义 Kafka 接收器的交付语义。有效的枚举是'at-least-once','exactly-once'和'none'。有关更多详细信息,请参阅[一致性保证](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#consistency-guarantees)。 + - 必选:否 + - 字段类型:String + - 默认值:at-least-once +- **sink.parallelism** + - 描述:定义 Kafka sink 操作符的并行性。默认情况下,并行度由框架使用与上游链式运算符相同的并行度确定。 + - 必选:否 + - 字段类型:Integer + - 默认值:无 + +## 五、数据类型 +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | +| --- | --- | +| 暂不支持 | 其他 | + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 + +### Sql +upsert-kafka +```sql +CREATE TABLE pageviews_per_region ( + id BIGINT, + col_bit BOOLEAN, + col_tinyint BIGINT, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'connector' = 'upsert-kafka-x', + 'topic' = 'pageviews_per_region_2', + 'properties.bootstrap.servers' = 'localhost:9092', + 'key.format' = 'json', + 'value.format' = 'json' +); + +CREATE TABLE pageviews ( + id BIGINT, + col_bit BOOLEAN, + col_tinyint BIGINT +) WITH ( + 'connector' = 'kafka-x', + 'topic' = 'pageviews_2', + 'properties.bootstrap.servers' = 'localhost:9092', + 'value.format' = 'debezium-json' +); + +-- 计算 pv、uv 并插入到 upsert-kafka sink +INSERT INTO pageviews_per_region +SELECT + id, + col_bit, + col_tinyint +FROM pageviews; + + +-- {"before":null,"after":{"id":1,"col_bit":true,"col_tinyint":1},"op":"c"} +-- {"before":{"id":1,"col_bit":true,"col_tinyint":1},"after":{"id":1,"col_bit":true,"col_tinyint":2},"op":"u"} +-- {"before":{"id":2,"col_bit":true,"col_tinyint":2},"after":null,"op":"d"} +``` diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-source.md" new file mode 100644 index 0000000000..3313f0cb0b --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kafka/kafka-source.md" @@ -0,0 +1,245 @@ + + +## 一、介绍 + +Kafka Source + +## 二、支持版本 + +kafka主流版本 + +## 三、插件名称 +| Sync | kafkasource、kafkareader | +| --- | --- | +| SQL | kafka-x | +| SQL(upsert) | upsert-kafka-x | + + +## 四、参数说明 + +### 1、Sync + +- **topic** + - 描述:要消费的topic,多个以,分割,当`mode`为`timestamp`、`specific-offsets`时不支持多topic + - 必选:是 + - 字段类型:String + - 默认值:无 +- **mode** + - 描述:kafka消费端启动模式,目前仅支持`kafkareader`插件 + - 可选值: + - group-offsets:     从ZK / Kafka brokers中指定的消费组已经提交的offset开始消费 + - earliest-offset:    从最早的偏移量开始(如果可能) + - latest-offset:      从最新的偏移量开始(如果可能) + - timestamp:         从每个分区的指定的时间戳开始 + - specific-offsets: 从每个分区的指定的特定偏移量开始 + - 必选:否 + - 字段类型:String + - 默认值:group-offsets +- **timestamp** + - 描述:指定的kafka时间戳采集起点,目前仅支持`kafkareader`插件 + - 必选:当`mode`为`timestamp`时必选 + - 字段类型:Long + - 默认值:无 +- **offset** + - 描述:消费的分区及对应的特定偏移量,目前仅支持`kafkareader`插件 + - 必选:当`mode`为`specific-offsets`时必选 + - 字段类型:String + - 格式:partition:0,offset:42;partition:1,offset:300;partition:2,offset:300 + - 默认值:无 +- **groupId** + - 描述:kafka消费组Id + - 必选:否 + - 字段类型:String + - 默认值:default +- **encoding** + - 描述:字符编码 + - 必选:否 + - 字段类型:String + - 默认值:UTF-8 +- **codec** + - 描述:编码解码器类型,支持 json、text + - text: + 将kafka获取到的消息字符串存储到一个key为message的map中,如:kafka中的消息为:{"key":"key","message":"value"}, + 则发送至下游的数据格式为: +```json +[ + { + "message":"{\"key\": \"key\", \"value\": \"value\"}" + } +] +``` + +- json:将kafka获取到的消息字符串按照json格式进行解析 + - 若该字符串为json格式 + - 当其中含有message字段时,发送至下游的数据格式为: +```json +[ + { + "key":"key", + "message":"value" + } +] +``` + + - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,发送至下游的数据格式为: +```json +[ + { + "key":"key", + "value":"value", + "message":"{\"key\": \"key\", \"value\": \"value\"}" + } +] +``` + + - 若改字符串不为json格式,则按照text类型进行处理 +- 必选:否 +- 字段类型:String +- 默认值:text +- **consumerSettings** + - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 + - 必选:是 + - 字段类型:Map + - 默认值:无 + - 如: +```json +{ + "consumerSettings":{ + "bootstrap.servers":"host1:9092,host2:9092,host3:9092" + } +} +``` + +- **column** + - 描述:kafka向MySQL写数据时,对应MySQL表中的字段名 + - 必选:否 + - 字段类型:List + - 默认值:无 + - 注意:需指定字段的具体信息,属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - 如: +```json +{ + "column": [ + { + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss" + } + ] +} +``` + + +### 2、SQL + +具体可以参考:[kafka-connector](https://ci.apache.org/projects/flink/flink-docs-release-1.12/dev/table/connectors/kafka.html) + +- **connector** + - 描述:kafka-x + - 必选:是 + - 字段类型:String + - 默认值:无 +- **topic** + - 描述:当表用作源时要从中读取数据的主题名称。它还通过用分号分隔主题来支持源的主题列表,如'topic-1;topic-2'. 请注意,只能为源指定“topic-pattern”和“topic”之一。当表用作接收器时,主题名称是要写入数据的主题。接收器不支持注意主题列表。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **topic-pattern** + - 描述:要从中读取的主题名称模式的正则表达式。当作业开始运行时,消费者将订阅名称与指定正则表达式匹配的所有主题。请注意,只能为源指定“topic-pattern”和“topic”之一。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **properties.bootstrap.servers** + - 描述:逗号分隔的 Kafka 代理列表。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **properties.group.id** + - 描述:Kafka source的消费组id,Kafka sink可选。 + - 必选:required by source + - 字段类型:String + - 默认值:无 +- **properties.*** + - 描述:这可以设置和传递任意 Kafka 配置。后缀名称必须与[Kafka 配置文档中](https://kafka.apache.org/documentation/#configuration)定义的配置键匹配。Flink 将删除“属性”。键前缀并将转换后的键和值传递给底层 KafkaClient。例如,您可以通过 禁用自动主题创建'properties.allow.auto.create.topics' = 'false'。但是有一些配置是不支持设置的,因为 Flink 会覆盖它们。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **format** + - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:此选项或'value.format'选项都是必需的。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **key.format** + - 描述:用用于反序列化和序列化 Kafka 消息关键部分的格式。有关更多详细信息和更多[格式](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)选项,请参阅格式页面。注意:如果定义了密钥格式,则该'key.fields' 选项也是必需的。否则 Kafka 记录将有一个空键。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **key.fields** + - 描述:定义表架构中物理列的显式列表,用于配置键格式的数据类型。默认情况下,此列表为空,因此未定义键。该列表应如下所示'field1;field2'。 + - 必选:否 + - 字段类型:List + - 默认值:无 +- **key.fields-prefix** + - 描述:为键格式的所有字段定义自定义前缀,以避免与值格式的字段发生名称冲突。默认情况下,前缀为空。如果定义了自定义前缀,则表架构 和'key.fields'都将使用前缀名称。在构造密钥格式的数据类型时,将删除前缀,并在密钥格式中使用非前缀名称。请注意,此选项要求'value.fields-include' 必须设置为'EXCEPT_KEY'。 + - 必选:否 + - 字段类型:String + - 默认值:无 +- **value.format** + - 描述:用于反序列化和序列化 Kafka 消息的值部分的格式。有关更多详细信息和更多格式选项,请参阅格式页面。注意:此选项或'format'[选项](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/formats/)都是必需的。 + - 必选:是 + - 字段类型:String + - 默认值:无 +- **value.fields-include** + - 描述:定义如何处理值格式的数据类型中的键列的策略。默认情况下,'ALL'表模式的物理列将包含在值格式中,这意味着键列出现在键和值格式的数据类型中 + - 必选:否 + - 字段类型:枚举 + - 可选的值:[ALL, EXCEPT_KEY] + - 默认值:ALL +- **scan.startup.mode** + - 描述:kafka消费的启动模式,有效值为'earliest-offset','latest-offset','group-offsets','timestamp'和'specific-offsets'。有关更多详细信息,请参阅以下[开始阅读位置](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#start-reading-position)。upsert模式此参数不生效,写死从earliest-offset处消费 + - 必选:否 + - 字段类型:String + - 默认值:group-offsets +- **scan.startup.specific-offsets** + - 描述:在'specific-offsets'启动模式下为每个分区指定偏移量,例如'partition:0,offset:42;partition:1,offset:300'. + - 必选:否 + - 字段类型:String + - 默认值:无 +- **scan.startup.timestamp-millis** + - 描述:从'timestamp'启动模式下使用的指定纪元时间戳(毫秒)开始。 + - 必选:否 + - 字段类型:Long + - 默认值:无 +- **scan.topic-partition-discovery.interval** + - 描述:消费者定期发现动态创建的 Kafka 主题和分区的时间间隔。 + - 必选:否 + - 字段类型:Duration + - 默认值:无 +- **sink.partitioner** + - 描述: 从 Flink 的分区到 Kafka 的分区的输出分区。有效值为 + - default: 使用 kafka 默认分区器对记录进行分区。 + - fixed:每个 Flink 分区最终最多包含一个 Kafka 分区。 + - round-robin:一个 Flink 分区被分发到 Kafka 分区粘性循环。它仅在未指定记录的键时有效。 + - 自定义FlinkKafkaPartitioner子类:例如'org.mycompany.MyPartitioner'. + - 有关更多详细信息,请参阅以下[接收器分区](https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/connectors/kafka.html#sink-partitioning)。 + - 必选:否 + - 字段类型:String + - 默认值:default +- **scan.parallelism** + - 描述:定义 Kafka sink 操作符的并行性。默认情况下,并行度由框架使用与上游链式运算符相同的并行度确定。 + - 必选:否 + - 字段类型:Integer + - 默认值:无 + +## 五、数据类型 +| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY、ARRAY、MAP、STRUCT、LIST、ROW | +| --- | --- | +| 暂不支持 | 其他 | + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/kingbase/kingbase-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-sink.md" similarity index 94% rename from docs/connectors/kingbase/kingbase-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-sink.md" index c2aee88341..d1d4d4b874 100644 --- a/docs/connectors/kingbase/kingbase-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-sink.md" @@ -24,18 +24,13 @@ KingBase v8 - 必选:是 - 参数类型:List - 默认值:无 - ```text - "connection": [ - { - "jdbcUrl": [ - "jdbc:kingbase8://localhost:54321/MOWEN" - ], - "table": [ - "type_test1" - ] - } - ] - ``` + ```json + "connection": [{ + "jdbcUrl":["jdbc:kingbase8://localhost:54321/MOWEN"], + "table": ["table"], + "schema":"public" + }] + ```
- **jdbcUrl** @@ -147,12 +142,19 @@ KingBase v8
- **url** - - 描述:jdbc:postgresql://localhost:5432/test + - 描述:jdbc:kingbase8://localhost:54321/MOWEN - 必选:是 - 参数类型:String - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 diff --git a/docs/connectors/kingbase/kingbase-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-source.md" similarity index 86% rename from docs/connectors/kingbase/kingbase-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-source.md" index b8f8c083eb..e57411cf4b 100644 --- a/docs/connectors/kingbase/kingbase-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kingbase/kingbase-source.md" @@ -23,13 +23,13 @@ KingBase v8 - 必选:是 - 参数类型:List - 默认值:无 - ```text - "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/MOWEN"], - "table": ["table"], - "schema":"public" + ```json + "connection": [{ + "jdbcUrl":["jdbc:kingbase8://localhost:54321/MOWEN"], + "table": ["table"], + "schema":"public" }] - ``` + ```
- **jdbcUrl** @@ -68,8 +68,8 @@ KingBase v8
- **fetchSize** - - 描述:一次性从数据库中读取多少条数据,默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2、连接参数useCursorFetch=true。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 描述:一次性从数据库中读取多少条数据,默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2,连接参数useCursorFetch=true。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 - 参数类型:int - 默认值:1024 @@ -116,16 +116,20 @@ KingBase v8 - **column** - 描述:需要读取的字段。 - - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash - "column":["*"] + - 格式:支持3种格式 +
+ 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ```json + "column":["*"] ``` 2.只指定字段名称: - ``` - "column":["id","name"] + + ```json + "column":["id","name"] ``` 3.指定具体信息: + ```json "column": [{ "name": "col", @@ -187,7 +191,7 @@ KingBase v8 ### 2、SQL - **connector** - - 描述:postgresql-x + - 描述:kingbase-x - 必选:是 - 参数类型:String - 默认值:无 @@ -200,6 +204,13 @@ KingBase v8 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -300,10 +311,10 @@ KingBase v8 ## 五、数据类型 -| 是否支持 | 类型 | -| --- | --- | +| 是否支持 | 类型名称 | +| --- |-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | -| 暂不支持 | ARRAY等 | +| 暂不支持 | ARRAY等 | ## 六、脚本示例 diff --git a/docs/connectors/kudu/kudu-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-lookup.md" similarity index 77% rename from docs/connectors/kudu/kudu-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-lookup.md" index 871cd2c040..8f25661bb0 100644 --- a/docs/connectors/kudu/kudu-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-lookup.md" @@ -2,8 +2,8 @@ ## 一、介绍 -Kudu维表,支持全量和异步方式
-全量缓存:将维表数据全部加载到内存中,建议数据量不大,且数据不经常变动的场景使用。
+Kudu维表,支持全量和异步方式 +全量缓存:将维表数据全部加载到内存中,建议数据量不大,且数据不经常变动的场景使用。 异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 ## 二、支持版本 @@ -22,89 +22,104 @@ kudu 常用版本 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **table-name** - 描述:要写入kudu表名 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **client.worker-count** - 描述:kudu worker的数量 - 必选:否 - 参数类型:int - 默认值:2 -
+ - **client.default-operation-timeout-ms** - 描述:kudu operation超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **client.default-admin-operation-timeout-ms** - 描述:admin operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **scan-token.query-timeout** - 描述:query operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **lookup.cache-type** - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU - 必选:否 - 参数类型:string - 默认值:LRU -
+ - **lookup.cache-period** - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) - 必选:否 - 参数类型:string - 默认值:3600000 -
+ - **lookup.cache.max-rows** - 描述:lru维表缓存数据的条数,默认10000条 - 必选:否 - 参数类型:string - 默认值:10000 -
+ - **lookup.cache.ttl** - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) - 必选:否 - 参数类型:string - 默认值:60000 -
-- **lookup.fetch-size** - - 描述:ALL维表每次从数据库加载的条数,默认1000条 + +- **lookup.parallelism** + - 描述:维表并行度 + - 必选:否 + - 参数类型:string + - 默认值:无 + + +- **scanner.batch-size-bytes** + - 描述:scanner在每批中返回的最大字节数 - 必选:否 - 参数类型:string - - 默认值:1000 -
+ - 默认值:0 + -- **lookup.parallelism** - - 描述:维表并行度 +- **scanner.limit** + - 描述:scanner将返回的行数限制 - 必选:否 - 参数类型:string - - 默认值:无 -
+ - 默认值:1000 + + +- **scanner.fault-tolerant** + - 描述:如果为真,如果当前服务器失败,则在另一台tablet服务器上恢复扫描 + - 必选:否 + - 参数类型:string + - 默认值:无 + ## 五、数据类型 -| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | +|是否支持 | 类型名称 | | --- | --- | +| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | | 暂不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git a/docs/connectors/kudu/kudu-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-sink.md" similarity index 84% rename from docs/connectors/kudu/kudu-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-sink.md" index 6908849071..4107e5a1cb 100644 --- a/docs/connectors/kudu/kudu-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-sink.md" @@ -6,7 +6,7 @@ kudu sink ## 二、支持版本 -主流版本 +Kudu 1.14.0 ## 三、插件名称 @@ -16,21 +16,21 @@ kudu sink ## 四、参数说明 -### 1、Sync +### 4.1、Sync - **masters** - 描述:kudu的IP地址和端口 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **table** - 描述:要写入kudu表名 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **column** - 描述:需要读取的字段。 @@ -45,75 +45,82 @@ kudu sink - 描述:写入 kudu 时,Kudu session的 flushMode - 必选:否 - 默认值:AUTO_FLUSH_SYNC -
+ + +- **writeMode** + - 描述:写入模式,可选值有insert、update、upsert、append + - 必选:否 + - 字段类型:string + - 默认值:append + - **maxBufferSize** - 描述:kudu client 中缓存数据的最大条数。 - 注意:当kudu session 中buffer里缓存的数据条数大于maxBufferSize,kudu session 会抛出"Buffer too big " 的异常,此异常并不会影响buffer中数据的实际写入, - 必选:否 - 默认值:1024 -
+ - **flushInterval** - 描述:批量写入 kudu 的刷新时间。 - 单位:毫秒 - 必选:否 - 默认值:10000 -
+ - **workerCount** - 描述:kudu worker的数量 - 必选:否 - 默认值:2 -
+ - **operationTimeout** - 描述:kudu operation超时时间 - 必选:否 - 默认值:30 * 1000(30秒) -
+ - **adminOperationTimeout** - 描述:admin operation 的超时时间 - 必选:否 - 默认值:30 * 1000(30秒) -
+ - **queryTimeout** - 描述:query operation 的超时时间 - 必选:否 - 默认值:30 * 1000(30秒) -
+ -### 2、SQL +### 4.2、SQL - **connector** - 描述:kudu-x - 必选:是 - 参数类型:string - 默认值:无 -
+ - **masters** - 描述:localhost:7051 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **table-name** - 描述:table-name - 必选:是 - 参数类型:string - 默认值:无 -
+ - **session.flush-mode** - 描述:写入 kudu 时,Kudu session的 flushMode - 必选:否 - 参数类型:string - 默认值:AUTO_FLUSH_SYNC -
+ - **session.mutation-buffer-space** - 描述:kudu client 中缓存数据的最大条数。 @@ -121,7 +128,7 @@ kudu sink - 必选:否 - 参数类型:int - 默认值:1024 -
+ - **sink.buffer-flush.interval** - 描述:批量写入 kudu 的刷新时间。 @@ -129,47 +136,62 @@ kudu sink - 必选:否 - 参数类型:int - 默认值:10000 -
- + + +- **sink.buffer-flush.max-rows** + - 描述:批量写入kudu的最大数据量。 + - 必选:否 + - 参数类型:int + - 默认值:1024 + + +- **sink.write-mode** + - 描述:写入模式,可选值insert、update、upsert。 + - 必选:否 + - 参数类型:string + - 默认值:insert + + - **client.worker-count** - 描述:kudu worker的数量 - 必选:否 - 参数类型:int - 默认值:2 -
+ - **client.default-operation-timeout-ms** - 描述:kudu operation超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **client.default-admin-operation-timeout-ms** - 描述:admin operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ -- **scan-token.query-timeout** +- **scan.query-timeout** - 描述:query operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **sink.parallelism** - 描述:sink并行度 - 必选:否 - 参数类型:string - 默认值:无 -
+ ## 五、数据类型 -| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | +|是否支持 | 类型名称 | | --- | --- | +| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | | 暂不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git a/docs/connectors/kudu/kudu-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-source.md" similarity index 91% rename from docs/connectors/kudu/kudu-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-source.md" index 9e2d09b004..f13e5b0210 100644 --- a/docs/connectors/kudu/kudu-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/kudu/kudu-source.md" @@ -1,10 +1,12 @@ +# Kudu Source + ## 一、介绍 读取kudu数据 ## 二、支持版本 -kudu主流版本 +Kudu 1.14.0 ## 三、插件名称 @@ -14,49 +16,49 @@ kudu主流版本 ## 四、参数说明 -#### 1、数据同步 +### 1、数据同步 - **masters** - 描述:kudu的IP地址和端口 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **table** - 描述:要写入kudu表名 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **workerCount** - 描述:kudu worker的数量 - 必选:否 - 参数类型:int - 默认值:2 -
+ - **operationTimeout** - 描述:kudu operation超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **adminOperationTimeout** - 描述:admin operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **queryTimeout** - 描述:query operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **column** - 描述:需要读取的字段。 @@ -67,6 +69,7 @@ kudu主流版本 - 字段类型:List - 默认值: + - **readMode** - 描述:kudu 读取模式 - 属性说明: kudu读取模式: @@ -83,8 +86,8 @@ kudu主流版本 - 默认值:READ_AT_SNAPSHOT -- **fetchSize** - - 描述:每次读取的数据条数,通过调整此参数来优化读取速率。默认为0代表kudu服务器自动选择合适的批量大小 +- **batchSizeBytes** + - 描述:scanner在每批中返回的最大字节数。 - 必选:否 - 字段类型:int - 默认值:0 @@ -96,54 +99,55 @@ kudu主流版本 - 字段类型:String - 默认值:无 -#### 2、SQL +### 2、SQL - **masters** - 描述:kudu的IP地址和端口 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **table-name** - 描述:要写入kudu表名 - 必选:是 - 参数类型:string - 默认值:无 -
+ - **client.worker-count** - 描述:kudu worker的数量 - 必选:否 - 参数类型:int - 默认值:2 -
+ - **client.default-operation-timeout-ms** - 描述:kudu operation超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **client.default-admin-operation-timeout-ms** - 描述:admin operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ - **scan-token.query-timeout** - 描述:query operation 的超时时间 - 必选:否 - 参数类型:int - 默认值:30 * 1000(30秒) -
+ ## 五、数据类型 -| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | +|是否支持 | 类型名称 | | --- | --- | +| 支持 | INT8、BYTE、INT16、SHORT、INT32、INT、INT64、LONG、FLOAT、DOUBLE、BOOLEAN、STRING、VARCHAR、DECIMAL、TIMESTAMP、DATE、TIME、BINARY | | 暂不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner-source.md" new file mode 100644 index 0000000000..938232f2ba --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner-source.md" @@ -0,0 +1,334 @@ +# LogMiner Source + +## 一、介绍 + +OracleLogMiner插件支持配置监听表名称以及读取起点读取日志数据。OracleLogMiner在checkpoint时保存当前消费的位点,因此支持续跑。 + +## 二、支持版本 + +oracle10,oracle11,oracle12,oracle19,支持RAC,主备架构 + +## 三、数据库配置 + +[Oracle配置LogMiner](LogMiner配置.md) + +## 四、LogMiner原理 + +[LogMiner原理](LogMiner原理.md) + +## 五、插件名称 + +| Sync | oraclelogminerreader、oraclelogminersource | +| --- | --- | +| SQL | oraclelogminer-x | + +## + +## 六、参数说明 + +### 1、Sync + +- **jdbcUrl** + - 描述:oracle数据库的JDBC URL链接 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **username** + - 描述:用户名 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **password** + - 描述:密码 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **table** + - 描述: 需要监听的表,格式为:schema.table,schema不能配置为*,但table可以配置*监听指定库下所有的表,如:["schema1.table1","schema1.table2","schema2.*"] + - 必选:否,不配置则监听除`SYS`库以外的所有库的所有表变更信息 + - 字段类型:数组 + - 默认值:无 + + +- **splitUpdate** + - 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见【七、数据结构说明】 + - 必选:否 + - 字段类型:boolean + - 默认值:false + + +- **cat** + - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 + - 必选:否 + - 字段类型:String + - 默认值:UPDATE,INSERT,DELETE + + +- **readPosition** + - 描述:Oracle实时采集的采集起点 + - 可选值: + - all: 从Oracle数据库中最早的归档日志组开始采集(不建议使用) + - current:从任务运行时开始采集 + - time: 从指定时间点开始采集 + - scn: 从指定SCN号处开始采集 + - 必选:否 + - 字段类型:String + - 默认值:current + + +- **startTime** + - 描述: 指定采集起点的毫秒级时间戳 + - 必选:当`readPosition`为`time`时,该参数必填 + - 字段类型:Long(毫秒级时间戳) + - 默认值:无 + + +- **startSCN** + - 描述: 指定采集起点的SCN号 + - 必选:当`readPosition`为`scn`时,该参数必填 + - 字段类型:String + - 默认值:无 + + +- **fetchSize** + - 描述: 批量从v$logmnr_contents视图中拉取的数据条数,对于大数据量的数据变更,调大该值可一定程度上增加任务的读取速度 + - 必选:否 + - 字段类型:Integer + - 默认值:1000 + + +- **queryTimeout** + - 描述: LogMiner执行查询SQL的超时参数,单位秒 + - 必选:否 + - 字段类型:Long + - 默认值:300 + + +- **supportAutoAddLog** + - 描述:启动LogMiner是否自动添加日志组(不建议使用) + - 必选:否 + - 字段类型:Boolean + - 默认值:false + + +- **pavingData** + - 描述:是否将解析出的json数据拍平,具体见【七、数据结构说明】 + - 必选:否 + - 字段类型:boolean + - 默认值:false + +### 2、SQL + +- **url** + - 描述:oracle数据库的JDBC URL链接 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **username** + - 描述:用户名 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **password** + - 描述:密码 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **table** + - 描述:需要解析的数据表。 + - 注意:SQL任务只支持监听单张表,且数据格式为schema.table + - 必选:否 + - 字段类型:string + - 默认值:无 + + +- **cat** + - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 + - 必选:否 + - 字段类型:String + - 默认值:UPDATE,INSERT,DELETE + + +- **read-position** + - 描述:Oracle实时采集的采集起点 + - 可选值: + - all: 从Oracle数据库中最早的归档日志组开始采集(不建议使用) + - current:从任务运行时开始采集 + - time: 从指定时间点开始采集 + - scn: 从指定SCN号处开始采集 + - 必选:否 + - 字段类型:String + - 默认值:current + + +- **start-time** + - 描述: 指定采集起点的毫秒级时间戳 + - 必选:当`readPosition`为`time`时,该参数必填 + - 字段类型:Long(毫秒级时间戳) + - 默认值:无 + + +- **start-scn** + - 描述: 指定采集起点的SCN号 + - 必选:当`readPosition`为`scn`时,该参数必填 + - 字段类型:String + - 默认值:无 + + +- **fetch-size** + - 描述: 批量从v$logmnr_contents视图中拉取的数据条数,对于大数据量的数据变更,调大该值可一定程度上增加任务的读取速度 + - 必选:否 + - 字段类型:Integer + - 默认值:1000 + + +- **query-timeout** + - 描述: LogMiner执行查询SQL的超时参数,单位秒 + - 必选:否 + - 字段类型:Long + - 默认值:300 + + +- **support-auto-add-log** + - 描述:启动LogMiner是否自动添加日志组(不建议使用) + - 必选:否 + - 字段类型:Boolean + - 默认值:false + + +- **io-threads** + - 描述:IO处理线程数,最大线程数为3 + - 必选:否 + - 字段类型:int + - 默认值:1 + + +- **max-log-file-size** + - 描述:logminer一次性加载的日志文件的大小,默认5g,单位byte + - 必选:否 + - 字段类型:long + - 默认值:5*1024*1024*1024 + + +- **transaction-cache-num-size** + - 描述:logminer可缓存DML的数量 + - 必选:否 + - 字段类型:long + - 默认值:800 + + +- **transaction-expire-time** + - 描述:logminer可缓存的失效时间,单位分钟 + - 必选:否 + - 字段类型:int + - 默认值:20 + +## 七、数据结构 + +在2021-06-29 23:42:19(时间戳:1624981339000)执行: + +```sql +INSERT INTO TIEZHU.RESULT1 ("id", "name", "age") +VALUES (1, 'a', 12) +``` + +在2021-06-29 23:42:29(时间戳:1624981349000)执行: + +```sql +UPDATE TIEZHU.RESULT1 t +SET t."id" = 2, + t."age" = 112 +WHERE t."id" = 1 +``` + +在2021-06-29 23:42:34(时间戳:1624981354000)执行: + +```sql + DELETE + FROM TIEZHU.RESULT1 + WHERE "id" = 2 +``` + +1、pavingData = true, splitUpdate = false RowData中的数据依次为: + +``` +//scn schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age +[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a", 12] +[49982969,"TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", 1, "a", 12 , 2, "a", 112] +[49982973,"TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a",112 , null, null, null] +``` + +2、pavingData = false, splitUpdate = false RowData中的数据依次为: + +``` +//scn, schema, table, ts, opTime, type, before, after +[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] +[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE", {"id":1, "name":"a", "age":12}, {"id":2, "name":"a", "age":112}] +[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] +``` + +3、pavingData = true, splitUpdate = true RowData中的数据依次为: + +``` +//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age +[49982945,"TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, null, null, 1, "a",12 ] + +//scn, schema, table, opTime, ts, type, before_id, before_name, before_age +[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", 1, "a", 12] +//scn, schema, table, opTime, ts, type, after_id, after_name, after_age +[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", 2, "a", 112] + +//scn, schema, table, ts, opTime, type, before_id, before_name, before_age, after_id, after_name, after_age +[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", 2, "a", 112, null, null, null] + + +``` + +4、pavingData = false, splitUpdate = true RowData中的数据依次为: + +``` +//scn, schema, table, ts, opTime, type, before, after +[49982945, "TIEZHU", "RESULT1", 6815665753853923328, "2021-06-29 23:42:19.0", "INSERT", null, {"id":1, "name":"a", "age":12}] +//scn, schema, table, ts, opTime, type, before +[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_BEFORE", {"id":1, "name":"a", "age":12}] +//scn, schema, table, ts, opTime, type, after +[49982969, "TIEZHU", "RESULT1", 6815665796098953216, "2021-06-29 23:42:29.0", "UPDATE_AFTER", {"id":2, "name":"a", "age":112}] +//scn, schema, table, ts, opTime, type, before, after +[49982973, "TIEZHU", "RESULT1", 6815665796140896256, "2021-06-29 23:42:34.0", "DELETE", {"id":2, "name":"a", "age":112}, null] + +``` + +- scn:Oracle数据库变更记录对应的scn号 +- type:变更类型,INSERT,UPDATE、DELETE +- opTime:数据库中SQL的执行时间 +- ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下: + +```java +long id=Long.parseLong("6815665753853923328"); + long res=id>>22; + DateFormat sdf=new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + System.out.println(sdf.format(res)); //2021-06-29 23:42:24 +``` + +## 八、数据类型 + +| 是否支持 | 数据类型 | +| --- | --- | +| 支持 | DATE,TIMESTAMP,TIMESTAMP WITH LOCAL TIME ZONE,TIMESTAMP WITH TIME ZONE, CHAR,NCHAR,NVARCHAR2,ROWID,VARCHAR2,VARCHAR,LONG,RAW,LONG RAW,INTERVAL YEAR,INTERVAL DAY,BLOB,CLOB,NCLOB, NUMBER,SMALLINT,INT INTEGER,FLOAT,DECIMAL,NUMERIC,BINARY_FLOAT,BINARY_DOUBLE | +| 不支持 | BFILE,XMLTYPE,Collections | + +## 九、脚本示例 + +见项目内`ChunJun : Local : Test`模块中的`demo`文件夹。 diff --git "a/docs/connectors/logminer/LogMiner\345\216\237\347\220\206.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\345\216\237\347\220\206.md" similarity index 97% rename from "docs/connectors/logminer/LogMiner\345\216\237\347\220\206.md" rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\345\216\237\347\220\206.md" index b91ee19f16..0338d709af 100644 --- "a/docs/connectors/logminer/LogMiner\345\216\237\347\220\206.md" +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\345\216\237\347\220\206.md" @@ -200,9 +200,7 @@ WHERE 查询出来的数据示例: -
- -
+![image](../../../website/src/images/doc/LogMiner/LogMiner23.png) # ChunJun如何使用Logminer @@ -212,7 +210,7 @@ WHERE - 找到需要分析的Redolog日志,加载到Logminer - 开启Logminer,在 v$LOGMNR_CONTENTS 查询感兴趣数据 ### 1. 查找RedoLog文件 -从上面介绍中 我们可以知道 Redolog来源于日志组和归档日志里,所以chunjun 根据SCN号查询日志组以及归档日志获取到对应的文件 +从上面介绍中 我们可以知道 Redolog来源于日志组和归档日志里,所以ChunJun 根据SCN号查询日志组以及归档日志获取到对应的文件 ```sql SELECT MIN(name) name, @@ -248,13 +246,13 @@ ORDER BY first_change# ``` 查询出来的数据示例: -
- -
+ +![image](../../../website/src/images/doc/LogMiner/LogMiner21.png) + + 注意: 如果Logminer的处理速度比Oracle产生数据速度快,那么理论上ChunJun只需要加载日志组文件不需要加载归档日志文件,而Logminer加载文件会比较消耗资源,所以会先进行RedoLog文件的查找,如果本次查找的文件和上次的没有区别,说明Logminer不需要加载新的日志文件,只需要重新再从视图里查询数据即可 - ### 2. 加载文件到Logminer 通过一个存储过程 查询到日志文件之后 加载到Logminer里 并开启Logminer ```sql @@ -327,10 +325,7 @@ WHERE ChunJun就是在一个循环里 执行上述sql语句查询数据。 查询日志文件,加载到logminer,开启logminer,读取数据,更新当前最新SCN号,当数据读取完毕,代表本次加载的日志文件加载完了,通过SCN号寻找后续日志文件,重复上述操作 -
- -
- +![image](../../../website/src/images/doc/LogMiner/LogMiner22.png) 从 v$logmnr_contents获取到数据之后,ChunJun 使用 net.sf.jsqlparser.parser.CCJSqlParserUtil 来解析 sql_redo 值 获取到的sql_redo语句格式示例: diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\351\205\215\347\275\256.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\351\205\215\347\275\256.md" new file mode 100644 index 0000000000..610f59b512 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/logminer/LogMiner\351\205\215\347\275\256.md" @@ -0,0 +1,491 @@ +# Oracle配置LogMiner + +注意: + +1、某个Oracle数据源能同时运行的任务数量取决于该Oracle的内存大小 + +2、若数据量太大导致日志组频繁切换需要增加日志组数量,增大单个日志组存储大小 + +## 一、Oracle 10g(单机版) + +### 1、查询Oracle版本信息,这里配置的是`Oracle 10g` + +```sql +--查看oracle版本 +select * +from v$version; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner1.png) + +本章Oracle的版本如上图所示。 + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 + +```sql +--查询数据库归档模式 +archive +log list; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner2.png) + +图中显示`No Archive Mode`表示未开启日志归档。 + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 + +#### a、配置归档日志保存的路径 + +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 + +```shell +# 创建归档日志保存目录 +mkdir -p /data/oracle/archivelog + +# 进入Oracle目录 +cd $ORACLE_HOME + +# 查看Oracle权限组,本章权限组如下图所示 +ls -l + +# 对归档日志保存目录赋予相应权限 +chown -R 下图中的用户名:下图中的组名 /data/oracle/ +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner3.png) +```sql +--配置归档日志保存的路径 +alter +system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` + +#### b、关闭数据库 + +```sql +shutdown +immediate; +startup +mount; +``` + +#### c、开启日志归档 + +```sql +--开启日志归档 +alter +database archivelog; +``` + +#### d、开启扩充日志 + +```sql +--开启扩充日志 +alter +database add supplemental log data (all) columns; +``` + +#### e、开启数据库 + +```sql +alter +database open; +``` + +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 + +![image](../../../website/src/images/doc/LogMiner/LogMiner4.png) + +### 4、配置日志组 + +#### a、查询默认日志组信息 + +```sql +SELECT * +FROM v$log; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner5.png) + +如上图所示,日志组的默认数量为2组,大小为4194304/1024/1024 = 4MB,这意味着日志大小每达到4MB就会进行日志组的切换,切换太过频繁会导致查询出错,因此需要增加日志组数量及大小。 + +#### b、查询日志组储存路径 + +```sql +SELECT * +FROM v$logfile; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner6.png) + +如上图所示,默认路径为`/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/`。 + +#### c、新增日志组与删除原有日志组 + +请与DBA联系,决定是否可以删除原有日志组。 + +```sql +--增加两组日志组 +alter +database add logfile group 3 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo3.log') size 200m; +alter +database add logfile group 4 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo4.log') size 200m; +``` + +```sql +--删除原有两组日志组,并继续新增两组日志组 +alter +system checkpoint; +alter +system switch logfile; +alter +database drop +logfile group 1; +alter +database drop +logfile group 2; +alter +database add logfile group 1 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo1.log') size 200m; +alter +database add logfile group 2 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo2.log') size 200m; +``` + +#### d、查询创建的日志组 + +```sql +SELECT * +FROM v$log; +SELECT * +FROM v$logfile; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner7.png) + +![image](../../../website/src/images/doc/LogMiner/LogMiner8.png) + +### 5、检查是否安装LogMiner工具 + +Oracle10g默认已安装LogMiner工具包,通过以下命令查询: + +```sql +desc DBMS_LOGMNR; +desc DBMS_LOGMNR_D; +``` + +若无信息打印,则执行下列SQL初始化LogMiner工具包: + +```sql +@ +$ORACLE_HOME +/rdbms/admin/dbmslm.sql; +@ +$ORACLE_HOME +/rdbms/admin/dbmslmd.sql; +``` + +### 6、创建LogMiner角色并赋权 + +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 + +```sql +create role roma_logminer_privs; +grant +create +session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +``` + +### 7、创建LogMiner用户并赋权 + +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 + +```sql +create +user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter +user roma_logminer quota unlimited on users; +``` + +### 8、验证用户权限 + +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: + +```sql + SELECT * + FROM USER_ROLE_PRIVS; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner9.png) + +```sql +SELECT * +FROM SESSION_PRIVS; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner10.png) + +至此,Oracle 10g数据库LogMiner实时采集配置完毕。 + +## 二、Oracle 11g(单机版) + +### 1、查询Oracle版本信息,这里配置的是`Oracle 11g` + +```sql +--查看oracle版本 +select * +from v$version; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner11.png) + +本章Oracle的版本如上图所示。 + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 + +```sql +--查询数据库归档模式 +archive +log list; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner12.png) + +图中显示`No Archive Mode`表示未开启日志归档。 + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 + +#### a、配置归档日志保存的路径 + +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 + +```sql + alter +system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` + +#### b、关闭数据库 + +```sql +shutdown +immediate; +startup +mount; +``` + +#### c、开启日志归档 + +```sql +--开启日志归档 +alter +database archivelog; +``` + +#### d、开启扩充日志 + +```sql +--开启扩充日志 +alter +database add supplemental log data (all) columns; +``` + +#### e、开启数据库 + +```sql +alter +database open; +``` + +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 +![image](../../../website/src/images/doc/LogMiner/LogMiner13.png) + +### 4、检查是否安装LogMiner工具 + +Oracle11g默认已安装LogMiner工具包,通过以下命令查询: + +```sql +desc DBMS_LOGMNR; +desc DBMS_LOGMNR_D; +``` + +若无信息打印,则执行下列SQL初始化LogMiner工具包: + +```sql +@ +$ORACLE_HOME +/rdbms/admin/dbmslm.sql; +@ +$ORACLE_HOME +/rdbms/admin/dbmslmd.sql; +``` + +### 5、创建LogMiner角色并赋权 + +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 + +```sql +create role roma_logminer_privs; +grant +create +session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +``` + +### 6、创建LogMiner用户并赋权 + +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 + +```sql +create +user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter +user roma_logminer quota unlimited on users; +``` + +### 7、验证用户权限 + +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: + +```sql + SELECT * + FROM USER_ROLE_PRIVS; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner14.png) + +```sql +SELECT * +FROM SESSION_PRIVS; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner15.png) + +至此,Oracle 11g数据库LogMiner实时采集配置完毕。 + +## 三、Oracle 12c(单机版非CBD) + +### 1、查询Oracle版本信息,这里配置的是`Oracle 12c` + +```sql +--查看oracle版本 +select BANNER +from v$version; +``` +![image](../../../website/src/images/doc/LogMiner/LogMiner16.png) + +本章Oracle的版本如上图所示。 + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 + +```sql +--查询数据库归档模式 +archive +log list; +``` +![image](../../../website/src/images/doc/LogMiner/LogMiner17.png) + +图中显示`No Archive Mode`表示未开启日志归档。 + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 + +#### a、配置归档日志保存的路径 + +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 + +```sql + alter +system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` + +#### b、关闭数据库 + +```sql +shutdown +immediate; +startup +mount; +``` + +#### c、开启日志归档 + +```sql +--开启日志归档 +alter +database archivelog; +``` + +#### d、开启扩充日志 + +```sql +--开启扩充日志 +alter +database add supplemental log data (all) columns; +``` + +#### e、开启数据库 + +```sql +alter +database open; +``` + +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 +![image](../../../website/src/images/doc/LogMiner/LogMiner18.png) +### 4、创建LogMiner角色并赋权 + +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 + +```sql +create role roma_logminer_privs; +grant +create +session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,logmining,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +grant LOGMINING to roma_logminer_privs; +``` + +### 5、创建LogMiner用户并赋权 + +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 + +```sql +create +user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter +user roma_logminer quota unlimited on users; +``` + +### 6、验证用户权限 + +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: + +```sql + SELECT * + FROM USER_ROLE_PRIVS; +``` + +![image](../../../website/src/images/doc/LogMiner/LogMiner19.png) + +```sql +SELECT * +FROM SESSION_PRIVS; +``` +![image](../../../website/src/images/doc/LogMiner/LogMiner20.png) + +至此,Oracle 12c数据库LogMiner实时采集配置完毕。 diff --git a/docs/connectors/mongodb/mongodb-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-lookup.md" similarity index 87% rename from docs/connectors/mongodb/mongodb-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-lookup.md" index 36bc73eed1..a0186e2ac0 100644 --- a/docs/connectors/mongodb/mongodb-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-lookup.md" @@ -86,8 +86,14 @@ MongoDB 3.4及以上 - 描述:维表并行度 - 必选:否 - 默认值:无 -## -## 五、脚本示例 + +## 五、数据类型 +| 是否支持 | 类型名称 | +| --- | --- | +| 支持 | long double decimal objectId string bindata date timestamp bool | +| 不支持 | array | + +## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/mongodb/mongodb-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-sink.md" similarity index 94% rename from docs/connectors/mongodb/mongodb-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-sink.md" index 59fbb16f40..b91c66d25a 100644 --- a/docs/connectors/mongodb/mongodb-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-sink.md" @@ -1,3 +1,8 @@ +--- +sidebar_position: 6 +title: MongoDB Sink +--- + ## 一、介绍 向MongoDB中写入数据 @@ -129,18 +134,10 @@ SQL计算暂时只支持INSERT模式,后续可加入如果配置主键则使 - 必选:否 - 默认值:无 ## 五、数据类型 -| 支持 | int | +| 是否支持 | 类型名称 | | --- | --- | -| | long | -| | double | -| | decimal | -| | objectId | -| | string | -| | bindata | -| | date | -| | timestamp | -| | bool | -| 暂不支持 | array | +| 支持 | long double decimal objectId string bindata date timestamp bool | +| 不支持 | array | ## 六、脚本示例 diff --git a/docs/connectors/mongodb/mongodb-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-source.md" similarity index 92% rename from docs/connectors/mongodb/mongodb-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-source.md" index d19b780a35..80881cdb3e 100644 --- a/docs/connectors/mongodb/mongodb-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mongodb/mongodb-source.md" @@ -90,18 +90,10 @@ MongoDB 3.4及以上 #### 2、SQL计算 暂不支持 ## 五、数据类型 -| 支持 | int | +| 是否支持 | 类型名称 | | --- | --- | -| | long | -| | double | -| | decimal | -| | objectId | -| | string | -| | bindata | -| | date | -| | timestamp | -| | bool | -| 暂不支持 | array | +| 支持 | long double decimal objectId string bindata date timestamp bool | +| 不支持 | array | ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/mysql/mysql-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-lookup.md" similarity index 77% rename from docs/connectors/mysql/mysql-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-lookup.md" index 051e8114d8..62d576d0f1 100644 --- a/docs/connectors/mysql/mysql-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-lookup.md" @@ -28,6 +28,13 @@ mysql5.x - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -92,7 +99,7 @@ mysql5.x
## 五、数据类型 -| 支持 | BOOLEAN、TINYINT、SMALLINT、INT、BIGINT、FLOAT、DOUBLE、DECIMAL、STRING、VARCHAR、CHAR、TIMESTAMP、DATE、BINARY | +| 支持 |BOOLEAN、BIT、TINYINT、TINYINT UNSIGNED、SMALLINT、SMALLINT UNSIGNED、MEDIUMINT、MEDIUMINT UNSIGNED、 INT、INT UNSIGNED、INTEGER、INT UNSIGNED、BIGINT、BIGINT UNSIGNED、REAL、FLOAT、FLOAT UNSIGNED、DECIMAL、DECIMAL UNSIGNED、NUMERIC、DOUBLE、DOUBLE UNSIGNED、STRING、VARCHAR、CHAR、TIMESTAMP 、DATETIME、DATE、TIME、YEAR、TINYBLOB、BLOB、MEDIUMBLOB、LONGBLOB、TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT、BINARY、VARBINARY、JSON、ENUM、SET、GEOMETRY | | --- | --- | | 暂不支持 | ARRAY、MAP、STRUCT、UNION | diff --git a/docs/connectors/oracle/oracle-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-sink.md" similarity index 56% rename from docs/connectors/oracle/oracle-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-sink.md" index 9abfc67b6e..a85ad40938 100644 --- a/docs/connectors/oracle/oracle-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-sink.md" @@ -1,49 +1,55 @@ -# Oracle Sink +# Mysql Sink ## 一、介绍 -oracle sink + +mysql sink ## 二、支持版本 -Oracle 9 及以上 +mysql5.x ## 三、插件名称 -| Sync | oraclesink、oraclewriter | -| --- | --- | -| SQL | oracle-x | +| Sync | mysqlsink、mysqlwriter | +| ---- | ---------------------- | +| SQL | mysql-x | ## 四、参数说明 + ### 1、Sync + - **connection** - - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 描述:数据库连接参数,包含 jdbcUrl、schema、table 等参数 - 必选:是 - 参数类型:List - 默认值:无 ```text "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], "table": ["table"], "schema":"public" }] ``` -
+
- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) + + - 描述:针对关系型数据库的 jdbc 连接字符串,jdbcUrl 参考文档:[MySQL 官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - 必选:是 - 参数类型:string - 默认值:无
- **schema** - - 描述:数据库schema名 + + - 描述:数据库 schema 名 - 必选:否 - 参数类型:string - - 默认值:用户名 + - 默认值:无
- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - 必选:是 - 参数类型:List @@ -51,6 +57,7 @@ Oracle 9 及以上
- **username** + - 描述:数据源的用户名 - 必选:是 - 参数类型:String @@ -58,6 +65,7 @@ Oracle 9 及以上
- **password** + - 描述:数据源指定用户名的密码 - 必选:是 - 参数类型:String @@ -65,13 +73,15 @@ Oracle 9 及以上
- **column** - - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + + - 描述:目的表需要写入数据的字段。例如: "column": [{"name":"id",type:"varchar"}] - 必选:是 - 参数类型:List - 默认值:无
-- **fullcolumn** +- **fullColumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 - 必选:否 - 参数类型:List @@ -79,6 +89,7 @@ Oracle 9 及以上
- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 - 参数类型:List @@ -86,65 +97,81 @@ Oracle 9 及以上
- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 - 参数类型:List - 默认值:无
-- **writeMode** - - 描述:控制写入数据到目标表采用 insert into 或者 merge into 语句 +- **mode** + + - 描述:控制写入数据到目标表采用 insert into 或者 replace into 或者 ON DUPLICATE KEY UPDATE 语句 - 必选:是 - - 所有选项:insert/update + - 所有选项:insert/replace/update - 参数类型:String - 默认值:insert
- **batchSize** - - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 + + - 描述:一次性批量提交的记录数大小,该值可以极大减少 ChunJun 与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成 ChunJun 运行进程 OOM 情况 - 必选:否 - 参数类型:int - - 默认值:1024 + - 默认值:1
- **updateKey** - - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + + - 描述:当写入模式为 update 和 replace 时,需要指定此参数的值为唯一索引字段 - 注意: - - 如果此参数为空,并且写入模式为update时,应用会自动获取数据库中的唯一索引; - - 如果数据表没有唯一索引,但是写入模式配置为update和,应用会以insert的方式写入数据; + - 如果此参数为空,并且写入模式为 update 和 replace 时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为 update 和 replace,应用会以 insert 的方式写入数据; - 必选:否 - - 参数类型:Map - - 示例:"updateKey": {"key": ["id"]} + - 参数类型:List< String > + - 示例:"updateKey": ["id"] - 默认值:无
- + - **semantic** - - 描述:sink端是否支持二阶段提交 + - 描述:sink 端是否支持二阶段提交 - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once + - 如果此参数为空,默认不开启二阶段提交,即 sink 端不支持 exactly_once 语义; + - 当前只支持 exactly-once 和 at-least-once - 必选:否 - 参数类型:String - 示例:"semantic": "exactly-once" - 默认值:at-least-once -
+
### 2、SQL + - **connector** - - 描述:oracle-x + + - 描述:mysql-x - 必选:是 - 参数类型:String - 默认值:无
- **url** - - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl + + - 描述:jdbc:mysql://localhost:3306/test - 必选:是 - 参数类型:String - 默认值:无
+- **schema** + + - 描述:数据库 schema 名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** + - 描述:表名 - 必选:是 - 参数类型:String @@ -152,6 +179,7 @@ Oracle 9 及以上
- **username** + - 描述:username - 必选:是 - 参数类型:String @@ -159,6 +187,7 @@ Oracle 9 及以上
- **password** + - 描述:password - 必选:是 - 参数类型:String @@ -166,6 +195,7 @@ Oracle 9 及以上
- **sink.buffer-flush.max-rows** + - 描述:批量写数据条数,单位:条 - 必选:否 - 参数类型:String @@ -173,6 +203,7 @@ Oracle 9 及以上
- **sink.buffer-flush.interval** + - 描述:批量写时间间隔,单位:毫秒 - 必选:否 - 参数类型:String @@ -180,37 +211,39 @@ Oracle 9 及以上
- **sink.all-replace** - - 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null) + + - 描述:是否全部替换数据库中的数据(如果数据库中原值不为 null,新值为 null,如果为 true 则会替换为 null) - 必选:否 - 参数类型:String - 默认值:false
- **sink.parallelism** + - 描述:写入结果的并行度 - 必选:否 - 参数类型:String - 默认值:无
- + - **sink.semantic** - - 描述:sink端是否支持二阶段提交 + - 描述:sink 端是否支持二阶段提交 - 注意: - - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; - - 当前只支持exactly-once 和at-least-once + - 如果此参数为空,默认不开启二阶段提交,即 sink 端不支持 exactly_once 语义; + - 当前只支持 exactly-once 和 at-least-once - 必选:否 - 参数类型:String - 示例:"semantic": "exactly-once" - 默认值:at-least-once -
+
## 五、数据类型 -| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | -| :---: | :---: | -| 暂不支持 | BFILE、XMLTYPE、Collections | -| 仅在 Sync 中支持 | BLOB、CLOB、NCLOB | -注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 +| 是否支持 | 类型名称 | +|:----:| :-------------------------------------------------------------------------------------------------------------: | +| 支持 |BOOLEAN、BIT、TINYINT、TINYINT UNSIGNED、SMALLINT、SMALLINT UNSIGNED、MEDIUMINT、MEDIUMINT UNSIGNED、 INT、INT UNSIGNED、INTEGER、INT UNSIGNED、BIGINT、BIGINT UNSIGNED、REAL、FLOAT、FLOAT UNSIGNED、DECIMAL、DECIMAL UNSIGNED、NUMERIC、DOUBLE、DOUBLE UNSIGNED、STRING、VARCHAR、CHAR、TIMESTAMP 、DATETIME、DATE、TIME、YEAR、TINYBLOB、BLOB、MEDIUMBLOB、LONGBLOB、TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT、BINARY、VARBINARY、JSON、ENUM、SET、GEOMETRY | +| 不支持 | ARRAY、MAP、STRUCT、UNION | ## 六、脚本示例 + 见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-source.md" new file mode 100644 index 0000000000..635d1f9351 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/mysql/mysql-source.md" @@ -0,0 +1,351 @@ +# Mysql Source + +## 一、介绍 + +支持从 mysql 离线读取,支持 mysql 实时间隔轮询读取 + +## 二、支持版本 + +mysql5.x + +## 三、插件名称 + +| Sync | mysqlsource、mysqlreader | +| ---- | ------------------------ | +| SQL | mysql-x | + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含 jdbcUrl、schema、table 等参数 + - 必选:是 + - 参数类型:List + - 默认值:无 + ```text + "connection": [{ + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useSSL=false"], + "table": ["table"], + "schema":"public" + }] + ``` +
+ +- **jdbcUrl** + + - 描述:针对关系型数据库的 jdbc 连接字符串,jdbcUrl 参考文档:[MySQL 官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **schema** + + - 描述:数据库 schema 名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ +- **table** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **fetchSize** + + - 描述:一次性从数据库中读取多少条数据,MySQL 默认一次将所有结果都读取到内存中,在数据量很大时可能会造成 OOM,设置这个参数可以控制每次读取 fetchSize 条数据,而不是默认的把所有数据一次读取出来;开启 fetchSize 需要满足:数据库版本要高于 5.0.2、连接参数 useCursorFetch=true。 + 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 参数类型:int + - 默认值:1024 +
+ +- **where** + + - 描述:筛选条件,reader 插件根据指定的 column、table、where 条件拼接 SQL,并根据这个 SQL 进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将 where 条件指定为 gmt_create > time。 + - 注意:不可以将 where 条件指定为 limit 10,limit 不是 SQL 的合法 where 子句。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **splitPk** + + - 描述:当 speed 配置中的 channel 大于 1 时指定此参数,Reader 插件根据并发数和此参数指定的字段拼接 sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐 splitPk 使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前 splitPk 仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun 将报错。 + - 如果 channel 大于 1 但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **queryTimeOut** + + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义 sql 查询时,可通过此参数指定超时时间。 + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ +- **customSql** + + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的 sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和 column 列表里的字段对应; + - 当指定了此参数时,connection 里指定的 table 无效; + - 当指定此参数时,column 必须指定具体字段信息,不能以\*号代替; + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:需要读取的字段。 + - 格式:支持 3 种格式 +
1. 读取全部字段,如果字段数量很多,可以使用下面的写法: + ```bash + "column":["*"] + ``` + 2. 只指定字段名称: + ``` + "column":["id","name"] + ``` + 3. 指定具体信息: + + ```JSON + "column": { + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + } + ``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把 value 的值作为常量列返回,如果指定的字段存在,当指定字段的值为 null 时,会以此 value 值作为默认值返回 + - 必选:是 + - 默认值:无 +
+ +- **polling** + + - 描述:是否开启间隔轮询,开启后会根据 pollingInterval 轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数 pollingInterval,increColumn,可以选择配置参数 startLocation。若不配置参数 startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的起始位置。 + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **pollingInterval** + + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为 5000 毫秒。 + - 必选:否 + - 参数类型:long + - 默认值:5000 +
+ +- **increColumn** + + - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在 column 中的顺序位置(从 0 开始) + - 必选:否 + - 参数类型:String 或 int + - 默认值:无 +
+ +- **startLocation** + + - 描述:增量查询起始位置 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **useMaxFunc** + + - 描述:用于标记是否保存 endLocation 位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为 true + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 参数类型:int + - 默认值:2 +
+ +### 2、SQL + +- **connector** + + - 描述:mysql-x + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **url** + + - 描述:jdbc:mysql://localhost:3306/test + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **schema** + + - 描述:数据库 schema 名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ +- **table-name** + + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ +- **username** + + - 描述:username + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:password + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **scan.polling-interval** + + - 描述:间隔轮训时间。非必填(不填为离线任务),无默认 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.parallelism** + + - 描述:并行度 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.fetch-size** + + - 描述:每次从数据库中 fetch 大小,单位:条。 + - 必选:否 + - 参数类型:String + - 默认值:1024 +
+ +- **scan.query-timeout** + + - 描述:数据库连接超时时间,单位:秒。 + - 必选:否 + - 参数类型:String + - 默认值:1 +
+ +- **scan.partition.column** + + - 描述:多并行度读取的切分字段 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.partition.strategy** + + - 描述:数据分片策略 + - 必选:否 + - 参数类型:String + - 默认值:range +
+ +- **scan.increment.column** + + - 描述:增量字段名称 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.increment.column-type** + - 描述:增量字段类型 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+- **scan.start-location** + + - 描述:增量字段开始位置,如果不指定则先同步所有,然后在增量 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.restore.columnname** + + - 描述:开启了 cp,任务从 sp/cp 续跑字段名称。如果续跑,则会覆盖 scan.start-location 开始位置,从续跑点开始 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.restore.columntype** + - 描述:开启了 cp,任务从 sp/cp 续跑字段类型 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +## 五、数据类型 + +| 是否支持 | 类型名称 | +| :------: | :-------------------------------------------------------------------------------------------------------------: | +| 支持 |BOOLEAN、BIT、TINYINT、TINYINT UNSIGNED、SMALLINT、SMALLINT UNSIGNED、MEDIUMINT、MEDIUMINT UNSIGNED、 INT、INT UNSIGNED、INTEGER、INT UNSIGNED、BIGINT、BIGINT UNSIGNED、REAL、FLOAT、FLOAT UNSIGNED、DECIMAL、DECIMAL UNSIGNED、NUMERIC、DOUBLE、DOUBLE UNSIGNED、STRING、VARCHAR、CHAR、TIMESTAMP 、DATETIME、DATE、TIME、YEAR、TINYBLOB、BLOB、MEDIUMBLOB、LONGBLOB、TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT、BINARY、VARBINARY、JSON、ENUM、SET、GEOMETRY | +| 不支持 | ARRAY、MAP、STRUCT、UNION | + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-lookup.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-lookup.md" new file mode 100644 index 0000000000..35e23a552d --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-lookup.md" @@ -0,0 +1,120 @@ +# Oracle Lookup + +## 一、介绍 + +Oracle维表,支持全量和异步方式
+全量缓存:将维表数据全部加载到内存中,建议数据量不大使用。
+异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 + +## 二、支持版本 + +Oracle 9 及以上 + + +## 三、插件名称 + +| SQL | oracle-x | +| ---- | -------- | + +## 四、参数说明 + +- **connector** + - 描述:oracle-x + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **url** + - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:String + - 默认值:用户名: +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ +- **username** + - 描述:username + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + - 描述:password + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **lookup.cache-type** + - 描述:维表缓存类型(NONE、LRU、ALL),默认LRU + - 必选:否 + - 参数类型:string + - 默认值:LRU +
+ +- **lookup.cache-period** + - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) + - 必选:否 + - 参数类型:string + - 默认值:3600000 +
+ +- **lookup.cache.max-rows** + - 描述:lru维表缓存数据的条数,默认10000条 + - 必选:否 + - 参数类型:string + - 默认值:10000 +
+ +- **lookup.cache.ttl** + - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) + - 必选:否 + - 参数类型:string + - 默认值:60000 +
+ +- **lookup.fetch-size** + - 描述:ALL维表每次从数据库加载的条数,默认1000条 + - 必选:否 + - 参数类型:string + - 默认值:1000 +
+ +- **lookup.parallelism** + - 描述:维表并行度 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ +## 五、数据类型 + +| 是否支持 | 类型名称 | +|:-----:| :----------------------------------------------------------: | +| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | +| 不支持 | BFILE、XMLTYPE、Collections、BLOB、CLOB、NCLOB | + + + +注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 + + + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-sink.md" new file mode 100644 index 0000000000..48b343cbf7 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-sink.md" @@ -0,0 +1,254 @@ +# Oracle Sink + +## 一、介绍 + +oracle sink + +## 二、支持版本 + +Oracle 9 及以上 + + +## 三、插件名称 + +| Sync | oraclesink、oraclewriter | +| ---- | ------------------------ | +| SQL | oracle-x | + + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], + "table": ["table"], + "schema":"public" + }] + ``` + +
+ +- **jdbcUrl** + + - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) + - 必选:是 + - 参数类型:string + - 默认值:无 +
+ +- **schema** + + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:用户名 +
+ +- **table** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **fullcolumn** + + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **preSql** + + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **postSql** + + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 参数类型:List + - 默认值:无 +
+ +- **writeMode** + + - 描述:控制写入数据到目标表采用 insert into 或者 merge into 语句 + - 必选:是 + - 所有选项:insert/update + - 参数类型:String + - 默认值:insert +
+ +- **batchSize** + + - 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况 + - 必选:否 + - 参数类型:int + - 默认值:1024 +
+ +- **updateKey** + + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和,应用会以insert的方式写入数据; + - 必选:否 + - 参数类型:Map + - 示例:"updateKey": {"key": ["id"]} + - 默认值:无 +
+ +- **semantic** + + - 描述:sink端是否支持二阶段提交 + - 注意: + - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; + - 当前只支持exactly-once 和at-least-once + - 必选:否 + - 参数类型:String + - 示例:"semantic": "exactly-once" + - 默认值:at-least-once +
+ +### 2、SQL + +- **connector** + - 描述:oracle-x + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **url** + - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:String + - 默认值:用户名: +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ +- **username** + - 描述:username + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + - 描述:password + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **sink.buffer-flush.max-rows** + - 描述:批量写数据条数,单位:条 + - 必选:否 + - 参数类型:String + - 默认值:1024 +
+ +- **sink.buffer-flush.interval** + - 描述:批量写时间间隔,单位:毫秒 + - 必选:否 + - 参数类型:String + - 默认值:10000 +
+ +- **sink.all-replace** + - 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null) + - 必选:否 + - 参数类型:String + - 默认值:false +
+ +- **sink.parallelism** + - 描述:写入结果的并行度 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **sink.semantic** + - 描述:sink端是否支持二阶段提交 + - 注意: + - 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义; + - 当前只支持exactly-once 和at-least-once + - 必选:否 + - 参数类型:String + - 示例:"semantic": "exactly-once" + - 默认值:at-least-once +
+ +## 五、数据类型 + + +| 是否支持 | 类型名称 | +|:-------------:| :----------------------------------------------------------: | +| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | +| 不支持 | BFILE、XMLTYPE、Collections | +| 仅在 Sync 中支持 | BLOB、CLOB、NCLOB | + + +注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 + diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-source.md" new file mode 100644 index 0000000000..735b793294 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/oracle/oracle-source.md" @@ -0,0 +1,357 @@ +# Oracle Source + +## 一、介绍 + +支持从oracle离线读取 + +## 二、支持版本 + +Oracle 9 及以上 + + +## 三、插件名称 + +| Sync | oraclesource、oraclereader | +| ---- | -------------------------- | +| SQL | oracle-x | + + +## 四、参数说明 + +### 1、Sync + +- **connection** + + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + + - 必选:是 + + - 参数类型:List + + - 默认值:无 + + ```text + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:orcl"], + "table": ["table"], + "schema":"public" + }] + ``` + +
+ +- **jdbcUrl** + + - 描述:针对关系型数据库的jdbc连接字符串,jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) + - 必选:是 + - 参数类型:string + - 默认值:用户名 +
+ +- **schema** + + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ +- **table** + + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 参数类型:List + - 默认值:无 +
+ +- **username** + + - 描述:数据源的用户名 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + + - 描述:数据源指定用户名的密码 + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **fetchSize** + + - 描述:一次性从数据库中读取多少条数据,ORACLE默认fetchSize大小为10。当fetchSize设置过小时导致频繁读取数据会影响查询速度,以及数据库压力。当fetchSize设置过大时在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 参数类型:int + - 默认值:1024 +
+ +- **where** + + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **splitPk** + + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,ChunJun将报错。 + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **queryTimeOut** + + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 参数类型:int + - 默认值:1000 +
+ +- **customSql** + + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **column** + + - 描述:需要读取的字段。 + + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ```bash + "column":["*"] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + - 属性说明: + + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + - 必选:是 + + - 默认值:无 +
+ +- **polling** + + - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的起始位置。 + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **pollingInterval** + + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 参数类型:long + - 默认值:5000 +
+ +- **increColumn** + + - 描述:增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 参数类型:String或int + - 默认值:无 +
+ +- **startLocation** + + - 描述:增量查询起始位置 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **useMaxFunc** + + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 参数类型:Boolean + - 默认值:false +
+ +- **requestAccumulatorInterval** + + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 参数类型:int + - 默认值:2 +
+ +### 2、SQL + +- **connector** + - 描述:oracle-x + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **url** + - 描述:jdbc:oracle:thin:@0.0.0.1:1521:orcl + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:String + - 默认值:用户名: +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 参数类型:String + - 默认值:无: +
+ +- **username** + - 描述:username + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **password** + - 描述:password + - 必选:是 + - 参数类型:String + - 默认值:无 +
+ +- **scan.polling-interval** + - 描述:间隔轮训时间。非必填(不填为离线任务),无默认 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.parallelism** + - 描述:并行度 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.fetch-size** + - 描述:每次从数据库中fetch大小,单位:条。 + - 必选:否 + - 参数类型:String + - 默认值:1024 +
+ +- **scan.query-timeout** + - 描述:数据库连接超时时间,单位:秒。 + - 必选:否 + - 参数类型:String + - 默认值:1 +
+ +- **scan.partition.column** + - 描述:多并行度读取的切分字段,多并行度下必需要设置 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.partition.strategy** + - 描述:数据分片策略 + - 必选:否 + - 参数类型:String + - 默认值:range +
+ +- **scan.increment.column** + - 描述:增量字段名称 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.increment.column-type** + - 描述:增量字段类型 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.start-location** + - 描述:增量字段开始位置,如果不指定则先同步所有,然后在增量 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.restore.columnname** + - 描述:开启了cp,任务从sp/cp续跑字段名称。如果续跑,则会覆盖scan.start-location开始位置,从续跑点开始 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +- **scan.restore.columntype** + - 描述:开启了cp,任务从sp/cp续跑字段类型 + - 必选:否 + - 参数类型:String + - 默认值:无 +
+ +## 五、数据类型 + +| 是否支持 | 类型名称 | +|:-------------:| :----------------------------------------------------------: | +| 支持 | SMALLINT、BINARY_DOUBLE、CHAR、VARCHAR、VARCHAR2、NCHAR、NVARCHAR2、INT、INTEGER、NUMBER、DECIMAL、FLOAT、DATE、RAW、LONG RAW、BINARY_FLOAT、TIMESTAMP、TIMESTAMP WITH LOCAL TIME ZONE、TIMESTAMP WITH TIME ZON、INTERVAL YEAR、INTERVAL DAY | +| 不支持 | BFILE、XMLTYPE、Collections | +| 仅在 Sync 中支持 | BLOB、CLOB、NCLOB | + + +注意:由于 flink DecimalType 的 PRECISION(1~38) 与 SCALE(0~PRECISION) 限制,oracle 的数值类型的数据在转换时可能会丢失精度 + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/pgwal/Postgres-CDC.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/pgwal/Postgres-CDC.md" similarity index 62% rename from docs/connectors/pgwal/Postgres-CDC.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/pgwal/Postgres-CDC.md" index 06421eed3b..cc5279bbd7 100644 --- a/docs/connectors/pgwal/Postgres-CDC.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/pgwal/Postgres-CDC.md" @@ -1,75 +1,64 @@ -# MySQL Postgres CDC Source +# Postgres CDC Source +## 一、介绍 - - -- [一、介绍](#一介绍) -- [二、支持版本](#二支持版本) -- [三、插件名称](#三插件名称) -- [四、数据库配置](#四数据库配置) - - [1 、修改配置文件](#1修改配置文件) - - [2 、添加权限](#2添加权限) -- [五、参数说明](#五参数说明) - - [1 、Sync](#1sync) - - [2 、SQL](#2sql) -- [六、数据结构](#六数据结构) -- [七、数据类型](#七数据类型) -- [八、脚本示例](#八脚本示例) - - - -## 一、介绍 Postgres CDC 插件实时地从Postgres中捕获变更数据。目前sink插件暂不支持数据还原,只能写入变更的日志数据。 -## 二、支持版本 +## 二、支持版本 + Postgres 10.0以上 ## 三、插件名称 + | Sync | pgwalsource、pgwalreader | | --- | --- | | SQL | pgwal-x | -## 四、数据库配置 -### 1、修改配置文件 +## 四、数据库配置 + +### 1、修改配置文件 + pgwal_format需要修改为 ROW 格式,在/etc/my.cnf文件里[mysqld]下添加下列配置 + ```sql -server_id=109 +server_id +=109 log_bin = /var/lib/mysql/mysql-bin binlog_format = ROW expire_logs_days = 30 ``` +## 五、参数说明 -## 五、参数说明 -### 1、Sync +### 1、Sync - **url** - 描述:数据库的jdbc连接字符串 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **username** - 描述:数据源的用户名 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **password** - 描述:数据源指定用户名的密码 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **databaseName** - 描述:数据源数据库名称 - 必选:是 - 字段类型:string - 默认值:无 -
+ - **tableList** - 描述:需要解析的数据表。 @@ -77,60 +66,56 @@ expire_logs_days = 30 - 必选:否 - 字段类型:list - 默认值:无 -
- + + - **slotName** - 描述:slot 名称 - 必选:否 - 字段类型:String - 默认值:true -
+ - **allowCreated** - 描述:是否自动创建slot - 必选:否 - 字段类型:boolean - 默认值:false -
+ - **temporary** - 描述:是否是临时的slot - 必选:否 - 字段类型:boolean - 默认值:false -
- + + - **statusInterval** - 描述:心跳间隔 - 必选:否 - 字段类型:int - 默认值:10 -
+ - **lsn** - 描述:位点信息 - 必选:否 - 字段类型:long - 默认值:0 -
+ - **slotAvailable** - 描述:slot是否可用 - 必选:否 - 字段类型:boolean - 默认值:false -
+## 六、数据类型 -## 六、数据类型 -| 支持 | BIT | +| 是否支持 | 数据类型 | | --- | --- | -| | NULL、 BOOLEAN、 TINYINT、 SMALLINT、 INTEGER、 INTERVAL_YEAR_MONTH、 BIGINT| -| | INTERVAL_DAY_TIME、 DATE、 TIME_WITHOUT_TIME_ZONE | -| | TIMESTAMP_WITHOUT_TIME_ZONE、 TIMESTAMP_WITH_LOCAL_TIME_ZONE、 FLOAT | -| | DOUBLE、 CHAR、 VARCHAR、 DECIMAL、 BINARY、 VARBINARY | -| 暂不支持 | 无 | +| 支持 |NULL, BOOLEAN, TINYINT, SMALLINT, INTEGER, INTERVAL_YEAR_MONTH, BIGINT, INTERVAL_DAY_TIME, DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_LOCAL_TIME_ZONE, FLOAT, DOUBLE, CHAR, VARCHAR, DECIMAL, BINARY, VARBINARY | +| 不支持 | None | +## 七、脚本示例 -## 七、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/postgres/postgres-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-lookup.md" similarity index 92% rename from docs/connectors/postgres/postgres-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-lookup.md" index 89b8921500..27eed08cce 100644 --- a/docs/connectors/postgres/postgres-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-lookup.md" @@ -28,6 +28,13 @@ PostgreSql 9.4及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -92,9 +99,10 @@ PostgreSql 9.4及以上
## 五、数据类型 +| 是否支持 | 数据类型 | +| --- | ---| | 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | -| --- | --- | -| 暂不支持 | ARRAY等 | +| 不支持 | ARRAY等 | ## 六、脚本示例 diff --git a/docs/connectors/postgres/postgres-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-sink.md" similarity index 96% rename from docs/connectors/postgres/postgres-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-sink.md" index 181e9a48f6..cfa209c56f 100644 --- a/docs/connectors/postgres/postgres-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-sink.md" @@ -20,13 +20,14 @@ PostgreSql 9.4及以上 - 必选:是 - 参数类型:List - 默认值:无 - ```text + + ```json "connection": [{ "jdbcUrl": ["jdbc:postgresql://0.0.0.1:5432/database?useSSL=false"], "table": ["table"], "schema":"public" }] - ``` + ```
- **jdbcUrl** @@ -144,6 +145,13 @@ PostgreSql 9.4及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -207,9 +215,10 @@ PostgreSql 9.4及以上 ## 五、数据类型 +| 是否支持 | 数据类型 | +| --- | ---| | 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | -| --- | --- | -| 暂不支持 | ARRAY等 | +| 不支持 | ARRAY等 | ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/postgres/postgres-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-source.md" similarity index 96% rename from docs/connectors/postgres/postgres-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-source.md" index 3c776a9e39..95c57214b9 100644 --- a/docs/connectors/postgres/postgres-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/postgresql/postgres-source.md" @@ -19,13 +19,14 @@ PostgreSql 9.4及以上 - 必选:是 - 参数类型:List - 默认值:无 - ```text + + ```json "connection": [{ "jdbcUrl": ["jdbc:postgresql://0.0.0.1:5432/database?useSSL=false"], "table": ["table"], "schema":"public" }] - ``` + ```
- **jdbcUrl** @@ -64,8 +65,8 @@ PostgreSql 9.4及以上
- **fetchSize** - - 描述:一次性从数据库中读取多少条数据,默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2、连接参数useCursorFetch=true。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 描述:一次性从数据库中读取多少条数据,默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据,而不是默认的把所有数据一次读取出来;开启fetchSize需要满足:数据库版本要高于5.0.2,连接参数useCursorFetch=true。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 - 参数类型:int - 默认值:1024 @@ -114,14 +115,17 @@ PostgreSql 9.4及以上 - 描述:需要读取的字段。 - 格式:支持3种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash + + ```json "column":["*"] ``` 2.只指定字段名称: - ``` + + ```json "column":["id","name"] ``` 3.指定具体信息: + ```json "column": [{ "name": "col", @@ -196,6 +200,13 @@ PostgreSql 9.4及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -295,9 +306,10 @@ PostgreSql 9.4及以上
## 五、数据类型 +| 是否支持 | 数据类型 | +| --- | ---| | 支持 | SMALLINT、SMALLSERIAL、INT2、INT、INTEGER、SERIAL、INT4、BIGINT、BIGSERIAL、OID、INT8、REAL、FLOAT4、FLOAT、DOUBLE PRECISION、FLOAT8、DECIMAL、NUMERIC、 CHARACTER VARYING、VARCHAR、CHARACTER、CHAR、TEXT、NAME、BPCHAR、BYTEA、TIMESTAMP、TIMESTAMPTZ、DATE、TIME、TIMETZ、 BOOLEAN、BOOL | -| --- | --- | -| 暂不支持 | ARRAY等 | +| 不支持 | ARRAY等 | ## 六、脚本示例 diff --git a/docs/connectors/saphana/saphana-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-sink.md" similarity index 87% rename from docs/connectors/saphana/saphana-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-sink.md" index 7606a2dfa1..c5e19b8bc2 100644 --- a/docs/connectors/saphana/saphana-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-sink.md" @@ -144,6 +144,13 @@ SAP HANA 2.0及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -205,10 +212,14 @@ SAP HANA 2.0及以上
## 五、数据类型 -| 支持 | DATE、TIME、SECONDDATE、TIMESTAMP、TINYINT、SMALLINT、INTEGER、BIGINT、DECIMAL、SMALLDECIMAL、REAL、DOUBLE、VARCHAR、NVARCHAR、ALPHANUM、SHORTTEXT、VARBINARY、BOOLEAN | -| :---: | :---: | -| 暂不支持 | ARRAY、ST_CircularString、ST_GeometryCollection、ST_LineString、ST_MultiLineString、ST_MultiPoint、ST_MultiPolygon、ST_Point、ST_Polygon等 | -| 仅在 Sync 中支持 | CLOB、NCLOB、TEXT、BINTEXT | + +| 是否支持 | 数据类型 | +| :--------------: | :----------------------------------------------------------: | +| 支持 | DATE、TIME、SECONDDATE、TIMESTAMP、TINYINT、SMALLINT、INTEGER、BIGINT、DECIMAL、SMALLDECIMAL、REAL、DOUBLE、VARCHAR、NVARCHAR、ALPHANUM、SHORTTEXT、VARBINARY、BOOLEAN | +| 暂不支持 | ARRAY、ST_CircularString、ST_GeometryCollection、ST_LineString、ST_MultiLineString、ST_MultiPoint、ST_MultiPolygon、ST_Point、ST_Polygon等 | +| 仅在 Sync 中支持 | CLOB、NCLOB、TEXT、BINTEXT | + + ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/saphana/saphana-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-source.md" similarity index 87% rename from docs/connectors/saphana/saphana-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-source.md" index ef43f60597..eb6a4c4151 100644 --- a/docs/connectors/saphana/saphana-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/saphana/saphana-source.md" @@ -113,23 +113,30 @@ SAP HANA 2.0及以上 - **column** - 描述:需要读取的字段。 - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: - ```bash - "column":["*"] - ``` - 2.只指定字段名称: - ``` - "column":["id","name"] - ``` - 3.指定具体信息: - ```json - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ```bash + "column":["*"] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + - 属性说明: - name:字段名称 - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 @@ -196,6 +203,13 @@ SAP HANA 2.0及以上 - 默认值:无
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -295,10 +309,12 @@ SAP HANA 2.0及以上
## 五、数据类型 -| 支持 | DATE、TIME、SECONDDATE、TIMESTAMP、TINYINT、SMALLINT、INTEGER、BIGINT、DECIMAL、SMALLDECIMAL、REAL、DOUBLE、VARCHAR、NVARCHAR、ALPHANUM、SHORTTEXT、VARBINARY、BOOLEAN | -| :---: | :---: | -| 暂不支持 | ARRAY、ST_CircularString、ST_GeometryCollection、ST_LineString、ST_MultiLineString、ST_MultiPoint、ST_MultiPolygon、ST_Point、ST_Polygon等 | -| 仅在 Sync 中支持 | CLOB、NCLOB、TEXT、BINTEXT | + +| 是否支持 | 数据类型 | +| :--------------: | :----------------------------------------------------------: | +| 支持 | DATE、TIME、SECONDDATE、TIMESTAMP、TINYINT、SMALLINT、INTEGER、BIGINT、DECIMAL、SMALLDECIMAL、REAL、DOUBLE、VARCHAR、NVARCHAR、ALPHANUM、SHORTTEXT、VARBINARY、BOOLEAN | +| 暂不支持 | ARRAY、ST_CircularString、ST_GeometryCollection、ST_LineString、ST_MultiLineString、ST_MultiPoint、ST_MultiPolygon、ST_Point、ST_Polygon等 | +| 仅在 Sync 中支持 | CLOB、NCLOB、TEXT、BINTEXT | diff --git a/docs/connectors/sqlserver/sqlserver-lookup.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-lookup.md" similarity index 90% rename from docs/connectors/sqlserver/sqlserver-lookup.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-lookup.md" index 4ff6739db7..87d4d65181 100644 --- a/docs/connectors/sqlserver/sqlserver-lookup.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-lookup.md" @@ -30,6 +30,13 @@ Microsoft SQL Server 2012及以上
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
+ - **table-name** - 描述:表名 - 必选:是 @@ -111,9 +118,10 @@ Microsoft SQL Server 2012及以上 ## 五、数据类型 -|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +|是否支持 | 类型名称 | | ---| ---| -| 暂不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | +|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +| 不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/sqlserver/sqlserver-sink.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-sink.md" similarity index 95% rename from docs/connectors/sqlserver/sqlserver-sink.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-sink.md" index 3b3c48cef9..aed69045fa 100644 --- a/docs/connectors/sqlserver/sqlserver-sink.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-sink.md" @@ -1,4 +1,4 @@ -# SqlServer Source +# SqlServer Sink ## 一、介绍 SqlServer Sink插件支持向SqlServer数据库写入数据 @@ -177,7 +177,14 @@ Microsoft SQL Server 2012及以上 - 字段类型:String - 默认值:无 -​
+
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 参数类型:string + - 默认值:无 +
- **table-name** - 描述:表名 @@ -255,8 +262,9 @@ Microsoft SQL Server 2012及以上 ## 五、数据类型 -|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +|是否支持 | 类型名称 | | ---| ---| -| 暂不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | +|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +| 不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | ## 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git a/docs/connectors/sqlserver/sqlserver-source.md "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-source.md" similarity index 89% rename from docs/connectors/sqlserver/sqlserver-source.md rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-source.md" index b3a872cacc..87239d553d 100644 --- a/docs/connectors/sqlserver/sqlserver-source.md +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlserver/sqlserver-source.md" @@ -19,16 +19,15 @@ Microsoft SQL Server 2012及以上

- **connection** - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - 必选:是 - - 字段类型:List - - 示例:指定jdbcUrl、schema、table -```json -"connection": [{ - "jdbcUrl": ["jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack"], - "table": ["table"], - "schema":"public" - }] -``` - + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack"], + "table": ["table"], + "schema":"public" + }] + ``` - 默认值:无 @@ -146,25 +145,24 @@ Microsoft SQL Server 2012及以上

- **column** - 描述:需要读取的字段。 - - 格式:支持3中格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: -```bash -"column":["*"] -``` - - - 2.只指定字段名称: -``` -"column":["id","name"] -``` - - - 3.指定具体信息: -```json -"column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] -``` + - 格式:支持3中格式
+ 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + ```bash + "column":["*"] + ``` + 2.只指定字段名称: + ``` + "column":["id","name"] + ``` + 3.指定具体信息: + ```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` - 属性说明: - name:字段名称 @@ -233,7 +231,7 @@ Microsoft SQL Server 2012及以上

### 2.SQL - **connector** - - 描述:connector type + - 描述:连接类型 - 必选:是 - 字段类型:String - 值:sqlserver-x @@ -248,6 +246,12 @@ Microsoft SQL Server 2012及以上


+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + - **table-name** - 描述:表名 - 必选:是 @@ -360,10 +364,10 @@ Microsoft SQL Server 2012及以上

## 五、数据类型 - -|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +|是否支持 | 类型名称 | | ---| ---| -| 暂不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | +|支持 | BIT、INT、SMALLINT、TINYINT、BIGINT、INT IDENTITY、REAL、FLOAT、DECIMAL、NUMERIC、CHAR、VARCHAR、VARCHAR(MAX)、TEXT、XML、NCHAR、NVARCHAR、NVARCHAR(MAX)、NTEXT、TIME、DATE、DATETIME、DATETIME2、SMALLDATETIME、DATETIMEOFFSET、TIMESTAMP、BINARY、VARBINARY、IMAGE、MONEY、SMALLMONEY、UNIQUEIDENTIFIER | +| 不支持 | CURSOR、ROWVERSION、HIERARCHYID、SQL_VARIANT、SPATIAL GEOMETRY TYPE、SPATIAL GEOGRAPHY TYPE、TABLE | # 六、脚本示例 见项目内`chunjun-examples`文件夹。 diff --git "a/docs/connectors/sqlservercdc/SqlserverCDC\345\216\237\347\220\206.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer CDC\345\256\236\346\227\266\351\207\207\351\233\206\345\216\237\347\220\206.md" similarity index 61% rename from "docs/connectors/sqlservercdc/SqlserverCDC\345\216\237\347\220\206.md" rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer CDC\345\256\236\346\227\266\351\207\207\351\233\206\345\216\237\347\220\206.md" index 840dcfdc16..a237da7362 100644 --- "a/docs/connectors/sqlservercdc/SqlserverCDC\345\216\237\347\220\206.md" +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer CDC\345\256\236\346\227\266\351\207\207\351\233\206\345\216\237\347\220\206.md" @@ -1,48 +1,35 @@ # SqlServer CDC实时采集原理 - - -- [SqlServer CDC实时采集原理](#sqlserver-cdc实时采集原理) -- [一、基础](#一基础) -- [二、配置](#二配置) -- [三、原理](#三原理) - - [1、SQL Server Agent](#1sql-server-agent) - - [2、数据库CDC开启前后对比](#2数据库cdc开启前后对比) - - [3、业务表CDC开启前后对比](#3业务表cdc开启前后对比) - - [4、采集原理](#4采集原理) - - [1、insert/delete](#1insertdelete) - - [2、update](#2update) - - [3、流程图](#3流程图) - - [4、数据格式](#4数据格式) - - -# 一、基础 + +## 一、基础 + SqlServer官方从SqlServer 2008版本开始支持CDC,文档连接如下: -[https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-ver15](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-ver15) +[about-change-data-capture-sql-server](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-ver15) +## 二、配置 -# 二、配置 配置文档链接如下: -[SqlServer配置CDC](../other/SqlserverCDC配置.md) +[SqlServer配置CDC](SqlServer配置CDC.md) + +## 三、原理 -# 三、原理 ### 1、SQL Server Agent -SQL Server Agent代理服务,是sql server的一个标准服务,作用是代理执行所有sql的自动化任务,以及数据库事务性复制等无人值守任务。这个服务在默认安装情况下是停止状态,需要手动启动,或改为自动运动,否则sql的自动化任务都不会执行的,还要注意服务的启动帐户。 -简单的说就是启动了这个服务,捕获进程才会处理事务日志并将条目写入CDC表。 -[https://docs.microsoft.com/zh-cn/sql/ssms/agent/sql-server-agent?view=sql-server-ver15](https://docs.microsoft.com/zh-cn/sql/ssms/agent/sql-server-agent?view=sql-server-ver15) +SQL Server Agent代理服务,是sql +server的一个标准服务,作用是代理执行所有sql的自动化任务,以及数据库事务性复制等无人值守任务。这个服务在默认安装情况下是停止状态,需要手动启动,或改为自动运动,否则sql的自动化任务都不会执行的,还要注意服务的启动帐户。 +简单的说就是启动了这个服务,捕获进程才会处理事务日志并将条目写入CDC表。 +[sql-server-agent](https://docs.microsoft.com/zh-cn/sql/ssms/agent/sql-server-agent?view=sql-server-ver15) ### 2、数据库CDC开启前后对比 + 开启前: -
- -
-
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver7.png) + 开启后: + EXEC sys.sp_cdc_enable_db; -
- -
-
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver8.png) 我们首先观察dbo下新增了一张**systranschemas**表,**systranschemas**表用于跟踪事务发布和快照发布中发布的项目中的架构更改。 @@ -53,13 +40,10 @@ EXEC sys.sp_cdc_enable_db; | endlsn | binary | 架构更改结束时的 LSN 值。 | | typeid | int | 架构更改的类型。 | - - 数据库下新增了名为cdc的schema,其实也新增了cdc用户。cdc下新增了以下四张表:
-**1、captured_columns** -为在捕获实例中跟踪的每一列返回一行。 默认情况下,将捕获源表中的所有列。 但是,如果为变更数据捕获启用了源表,则可以通过指定列列表将列包括在捕获范围内或排除在捕获范围之外。 -当没有任何业务表开启了CDC时,该表为空。 +* 1、captured_columns +为在捕获实例中跟踪的每一列返回一行。 默认情况下,将捕获源表中的所有列。 但是,如果为变更数据捕获启用了源表,则可以通过指定列列表将列包括在捕获范围内或排除在捕获范围之外。 当没有任何业务表开启了CDC时,该表为空。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -70,11 +54,8 @@ EXEC sys.sp_cdc_enable_db; | column_ordinal | int | 更改表中的列序号(从 1 开始)。 将排除更改表中的元数据列。 序号 1 将分配给捕获到的第一个列。 | | is_computed | bit | 表示捕获到的列是源表中计算所得的列。 | - - -**2、change_tables** -为数据库中的每个更改表返回一行。 对源表启用变更数据捕获时,将创建一个更改表。 -当没有任何业务表开启了CDC时,该表为空。 +* 2、change_tables +为数据库中的每个更改表返回一行。 对源表启用变更数据捕获时,将创建一个更改表。 当没有任何业务表开启了CDC时,该表为空。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -92,10 +73,8 @@ EXEC sys.sp_cdc_enable_db; | create_date | datetime | 启用源表的日期。 | | partition_switch | bit | 指示是否可以对启用了变更数据捕获的表执行 ALTER TABLE 的 SWITCH PARTITION 命令。 0 指示分区切换被阻止。 未分区表始终返回 1。 | - - -**3、ddl_history** -为对启用了变更数据捕获的表所做的每一项数据定义语言 (DDL) 更改返回一行。 可以使用此表来确定源表发生 DDL 更改的时间以及更改的内容。 此表中不包含未发生 DDL 更改的源表的任何条目。 +* 3、ddl_history +为对启用了变更数据捕获的表所做的每一项数据定义语言 (DDL) 更改返回一行。 可以使用此表来确定源表发生 DDL 更改的时间以及更改的内容。 此表中不包含未发生 DDL 更改的源表的任何条目。 当没有任何开启了CDC的业务表的表结构发生变更时,该表为空。 | 列名称 | 数据类型 | 说明 | @@ -107,11 +86,9 @@ EXEC sys.sp_cdc_enable_db; | ddl_lsn | binary(10) | 与 DDL 修改的提交相关联的日志序列号 (LSN)。 | | ddl_time | datetime | 对源表所做的 DDL 更改的日期和时间。 | - - -**4、index_columns** -为与更改表关联的每个索引列返回一行。 变更数据捕获使用这些索引列来唯一标识源表中的行。 默认情况下,将包括源表的主键列。 但是,如果在对源表启用变更数据捕获时指定了源表的唯一索引,则将改用该索引中的列。 如果启用净更改跟踪,则该源表需要主键或唯一索引。 -当没有任何开启了CDC的业务表存在存在索引列时,该表为空。 +* 4、index_columns +为与更改表关联的每个索引列返回一行。 变更数据捕获使用这些索引列来唯一标识源表中的行。 默认情况下,将包括源表的主键列。 但是,如果在对源表启用变更数据捕获时指定了源表的唯一索引,则将改用该索引中的列。 +如果启用净更改跟踪,则该源表需要主键或唯一索引。 当没有任何开启了CDC的业务表存在存在索引列时,该表为空。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -120,10 +97,8 @@ EXEC sys.sp_cdc_enable_db; | index_ordinal | tinyint | 索引中的列序号(从 1 开始)。 | | column_id | int | 源表中的列 ID。 | - - -**5、lsn_time_mapping** -为每个在更改表中存在行的事务返回一行。 该表用于在日志序列号 (LSN) 提交值和提交事务的时间之间建立映射。 没有对应的更改表项的项也可以记录下来, 以便表在变更活动少或者无变更活动期间将 LSN 处理的完成过程记录下来。 +* 5、lsn_time_mapping +为每个在更改表中存在行的事务返回一行。 该表用于在日志序列号 (LSN) 提交值和提交事务的时间之间建立映射。 没有对应的更改表项的项也可以记录下来, 以便表在变更活动少或者无变更活动期间将 LSN 处理的完成过程记录下来。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -132,13 +107,14 @@ EXEC sys.sp_cdc_enable_db; | tran_end_time | datetime | 事务结束的时间。 | | tran_id | varbinary (10) | 事务的 ID。 | - - cdc下新增以下函数:
-**1、fn_cdc_get_all_changes_** -为在指定日志序列号 (LSN) 范围内应用到源表的每项更改返回一行。 如果源行在该间隔内有多项更改,则每项更改都会表示在返回的结果集中。 除了返回更改数据外,四个元数据列还提供了将更改应用到另一个数据源所需的信息。 行筛选选项可控制元数据列的内容以及结果集中返回的行。 当指定“all”行筛选选项时,针对每项更改将只有一行来标识该更改。 当指定“all update old”选项时,更新操作会表示为两行:一行包含更新之前已捕获列的值,另一行包含更新之后已捕获列的值。此枚举函数是在对源表启用变更数据捕获时创建的。 函数名称是派生的,并使用 **cdc.fn_cdc_get_all_changes_**_capture_instance_ 格式,其中 _capture_instance_ 是在源表启用变更数据捕获时为捕获实例指定的值。 +* 1、fn_cdc_get_all_changes_ +为在指定日志序列号 (LSN) 范围内应用到源表的每项更改返回一行。 如果源行在该间隔内有多项更改,则每项更改都会表示在返回的结果集中。 除了返回更改数据外,四个元数据列还提供了将更改应用到另一个数据源所需的信息。 +行筛选选项可控制元数据列的内容以及结果集中返回的行。 当指定“all”行筛选选项时,针对每项更改将只有一行来标识该更改。 当指定“all update +old”选项时,更新操作会表示为两行:一行包含更新之前已捕获列的值,另一行包含更新之后已捕获列的值。此枚举函数是在对源表启用变更数据捕获时创建的。 函数名称是派生的,并使用 **cdc.fn_cdc_get_all_changes_**_ +capture_instance_ 格式,其中 _capture_instance_ 是在源表启用变更数据捕获时为捕获实例指定的值。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -146,20 +122,21 @@ cdc下新增以下函数: | __$seqval | binary(10) | 用于对某事务内的行更改进行排序的序列值。 | | __$operation | int | 标识将更改数据行应用到目标数据源所需的数据操作语言 (DML) 操作。 可以是以下值之一:
1 = 删除
2 = 插入
3 = 更新(捕获的列值是执行更新操作前的值)。 仅当指定了行筛选选项“all update old”时才应用此值。
4 = 更新(捕获的列值是执行更新操作后的值)。 | | __$update_mask | varbinary(128) | 位掩码,为捕获实例标识的每个已捕获列均对应于一个位。 当 __ $ operation = 1 或2时,该值将所有已定义的位设置为1。 当 __ $ operation = 3 或4时,只有与更改的列相对应的位设置为1。 | -| \ | 多种多样 | 函数返回的其余列是在创建捕获实例时标识的已捕获列。 如果已捕获列的列表中未指定任何列,则将返回源表中的所有列。 | - +| \ | 多种多样 | 函数返回的其余列是在创建捕获实例时标识的已捕获列。 如果已捕获列的列表中未指定任何列,则将返回源表中的所有列。 | - -**2、fn_cdc_get_net_changes_** +* 2、fn_cdc_get_net_changes_ 为 (LSN) 范围内的指定日志序列号内的每个源行返回一个净更改行,返回格式跟上面一样。 ### 3、业务表CDC开启前后对比 -开启前跟上一张图一致 +开启前跟上一张图一致 开启SQL: + ```sql -sys.sp_cdc_enable_table +sys +. +sp_cdc_enable_table -- 表所属的架构名 [ @source_schema = ] 'source_schema', @@ -167,7 +144,7 @@ sys.sp_cdc_enable_table [ @source_name = ] 'source_name' , -- 是用于控制更改数据访问的数据库角色的名称 -[ @role_name = ] 'role_name'。 +[ @role_name = ] 'role_name' -- 是用于命名变更数据捕获对象的捕获实例的名称,这个名称在后面的存储过程和函数中需要经常用到。 [,[ @capture_instance = ] 'capture_instance' ] @@ -189,16 +166,17 @@ sys.sp_cdc_enable_table -- allow_partition_switch 为 bit,默认值为 1。 [,[ @partition_switch = ] 'partition_switch' ] ``` + 开启后: -
- -
-
-此时,cdc下新增了一张名为dbo_kudu_CT的表,对于任意开启CDC的业务表而言,都会在其对应的cdc schema下创建一张格式为${schema}_${table}_CT的表。 +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver9.png) + +此时,cdc下新增了一张名为dbo_kudu_CT的表,对于任意开启CDC的业务表而言,都会在其对应的cdc schema下创建一张格式为$schema_$table}_CT的表。 **1、dbo_kudu_CT:** -对源表启用变更数据捕获时创建的更改表。 该表为对源表执行的每个插入和删除操作返回一行,为对源表执行的每个更新操作返回两行。 如果在启用源表时未指定更改表的名称,则会使用一个派生的名称。 名称的格式为 cdc。capture_instance _CT 其中 capture_instance 是源表的架构名称和格式 schema_table 的源表名称。 例如,如果对 AdventureWorks 示例数据库中的表 Person 启用了变更数据捕获,则派生的更改表名称将 cdc.Person_Address_CT。 +对源表启用变更数据捕获时创建的更改表。 该表为对源表执行的每个插入和删除操作返回一行,为对源表执行的每个更新操作返回两行。 如果在启用源表时未指定更改表的名称,则会使用一个派生的名称。 名称的格式为 +cdc。capture_instance _CT 其中 capture_instance 是源表的架构名称和格式 schema_table 的源表名称。 例如,如果对 AdventureWorks 示例数据库中的表 Person +启用了变更数据捕获,则派生的更改表名称将 cdc.Person_Address_CT。 | 列名称 | 数据类型 | 说明 | | --- | --- | --- | @@ -207,71 +185,54 @@ sys.sp_cdc_enable_table | __$seqval | binary(10) | 用于对事务内的行更改进行排序的序列值。 | | __$operation | int | 标识与相应更改关联的数据操作语言 (DML) 操作。 可以是以下值之一:
1 = 删除
2 = 插入
3 = 更新(旧值)列数据中具有执行更新语句之前的行值。
4 = 更新(新值)列数据中具有执行更新语句之后的行值。 | | __$update_mask | varbinary(128) | 基于更改表的列序号的位掩码,用于标识那些发生更改的列。 | -| \ | 多种多样 | 更改表中的其余列是在创建捕获实例时源表中标识为已捕获列的那些列。 如果已捕获列的列表中未指定任何列,则源表中的所有列将包括在此表中。 | +| \ | 多种多样 | 更改表中的其余列是在创建捕获实例时源表中标识为已捕获列的那些列。 如果已捕获列的列表中未指定任何列,则源表中的所有列将包括在此表中。 | | __ $ command_id | int | 跟踪事务中的操作顺序。 | - - **2、captured_columns:** -
- -
-
-**3、change_tables:** +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver10.png) -
- -
-
+**3、change_tables:** +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver11.png) ### 4、采集原理 + #### 1、insert/delete -对于insert和delete类型的数据变更,对于每一行变更都会在对应的${schema}_${table}_CT表中增加一行记录。对于insert,id,user_id,name记录的是insert之后的value值;对于delete,id,user_id,name记录的是delete之前的value值; -
- -
-
+ +对于insert和delete类型的数据变更,对于每一行变更都会在对应的${schema}_${table}_ +CT表中增加一行记录。对于insert,id,user_id,name记录的是insert之后的value值;对于delete,id,user_id,name记录的是delete之前的value值; +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver12.png) #### 2、update -a、更新了主键 -此时,SqlServer数据库的做法是在同一事物内,先将原来的记录删除,然后再重新插入。 -执行如下SQL,日志表如图所示: -UPDATE [dbo].[kudu] SET [id] = 2, [user_id] = '2', [name] = 'b' WHERE [id] = 1; -
- -
-
+ +a、更新了主键 此时,SqlServer数据库的做法是在同一事物内,先将原来的记录删除,然后再重新插入。 执行如下SQL,日志表如图所示: UPDATE [dbo].[kudu] SET [id] = 2, [user_id] = ' +2', [name] = 'b' WHERE [id] = 1; +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver13.png) + b、未更新主键 此时,SqlServer数据库的做法是直接更新字段信息。 执行如下SQL,日志表如图所示: UPDATE [dbo].[kudu] SET [user_id] = '3', [name] = 'c' WHERE [id] = 2; -
- -
-
- +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver14.png) #### 3、流程图 +![image](../../../website/src/images/doc/SqlserverCDC/SqlserverCdc流程图.png) -
- -
-
对于ChunJun SqlServer CDC实时采集插件,其基本原理便是以轮询的方式,循环调用fn_cdc_get_all_changes_函数,获取上次结束时的lsn与当前数据库最大lsn值之间的数据。对于insert/delete类型的数据获取并解析一行,对于update类型获取并解析两行。解析完成后把数据传递到下游并记录当前解析到的数据的lsn,为下次轮询做准备。 #### 4、数据格式 + ```json { - "type":"update", - "schema":"dbo", - "table":"tb1", - "lsn":"00000032:00002038:0005", - "ts": 6760525407742726144, - "before_id":1, - "after_id":2 + "type": "update", + "schema": "dbo", + "table": "tb1", + "lsn": "00000032:00002038:0005", + "ts": 6760525407742726144, + "before_id": 1, + "after_id": 2 } ``` diff --git "a/docs/connectors/sqlservercdc/SqlserverCDC\351\205\215\347\275\256.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer\351\205\215\347\275\256CDC.md" similarity index 54% rename from "docs/connectors/sqlservercdc/SqlserverCDC\351\205\215\347\275\256.md" rename to "docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer\351\205\215\347\275\256CDC.md" index cee0058106..2f17a0e4d6 100644 --- "a/docs/connectors/sqlservercdc/SqlserverCDC\351\205\215\347\275\256.md" +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlServer\351\205\215\347\275\256CDC.md" @@ -1,73 +1,59 @@ # SqlServer配置CDC - - -- [SqlServer配置CDC](#sqlserver配置cdc) - - [1、查询SqlServer数据库版本](#1查询sqlserver数据库版本) - - [2、查询当前用户权限,必须为 sysadmin 固定服务器角色的成员才允许对数据库启用CDC(变更数据捕获)功能](#2查询当前用户权限必须为 sysadmin 固定服务器角色的成员才允许对数据库启用cdc变更数据捕获功能) - - [3、查询数据库是否已经启用CDC(变更数据捕获)功能](#3查询数据库是否已经启用cdc变更数据捕获功能) - - [4、对数据库数据库启用CDC(变更数据捕获)功能](#4对数据库数据库启用cdc变更数据捕获功能) - - [5、查询表是否已经启用CDC(变更数据捕获)功能](#5查询表是否已经启用cdc变更数据捕获功能) - - [6、对表启用CDC(变更数据捕获)功能](#6对表启用cdc变更数据捕获功能) - - [7、确认CDC agent 是否正常启动](#7确认cdc-agent-是否正常启动) 注:SqlServer自2008版本开始支持CDC(变更数据捕获)功能,本文基于SqlServer 2017编写。 - - #### 1、查询SqlServer数据库版本 + SQL:`SELECT @@VERSION` 结果: -
- -
+![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver1.png) #### 2、查询当前用户权限,必须为 sysadmin 固定服务器角色的成员才允许对数据库启用CDC(变更数据捕获)功能 + SQL:`exec sp_helpsrvrolemember 'sysadmin'` 结果: -
- -
+![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver2.png) #### 3、查询数据库是否已经启用CDC(变更数据捕获)功能 + SQL:`select is_cdc_enabled, name from sys.databases where name = 'tudou'` 结果: -
- -
-
-0:未启用;1:启用 +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver3.png) + +0:未启用;1:启用 #### 4、对数据库数据库启用CDC(变更数据捕获)功能 + SQL: + ```sql -USE tudou +USE +tudou GO EXEC sys.sp_cdc_enable_db GO ``` - 重复第三步操作,确认数据库已经启用CDC(变更数据捕获)功能。 -
- -
- +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver4.png) #### 5、查询表是否已经启用CDC(变更数据捕获)功能 + SQL:`select name,is_tracked_by_cdc from sys.tables where name = 'test';` 结果: -
- -
-0:未启用;1:启用 +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver5.png) + +0:未启用;1:启用 #### 6、对表启用CDC(变更数据捕获)功能     + SQL: + ```sql EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @@ -75,49 +61,45 @@ EXEC sys.sp_cdc_enable_table @role_name = NULL, @supports_net_changes = 0; ``` -source_schema:表所在的schema名称 -source_name:表名 -role_name:访问控制角色名称,此处为null不设置访问控制 -supports_net_changes:是否为捕获实例生成一个净更改函数,0:否;1:是 +source_schema:表所在的schema名称 source_name:表名 role_name:访问控制角色名称,此处为null不设置访问控制 +supports_net_changes:是否为捕获实例生成一个净更改函数,0:否;1:是 重复第五步操作,确认表已经启用CDC(变更数据捕获)功能。 -
- -
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver6.png) 至此,表`test`启动CDC(变更数据捕获)功能配置完成。 #### 7、确认CDC agent 是否正常启动 + ```sql EXEC master.dbo.xp_servicecontrol N'QUERYSTATE', N'SQLSERVERAGENT' ``` -
- -
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver16.png) + 如显示上图状态,需要启动对应的agent. **Windows 环境操作开启 CDC agent** 点击 下图位置代理开启 -
- -
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver17.png)> **重新启动数据库** -
- -
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver18.png) **再次查询agent 状态,确认状态变更为running** -
- -
+ +![image](../../../website/src/images/doc/SqlserverCDC/Sqlserver19.png) 至此,表`test`启动CDC(变更数据捕获)功能配置完成。 **docker 环境操作开启 CDC agent** **开启mssql-server的代理服务**_ + ```shell docker exec -it sqlserver bash /opt/mssql/bin/mssql-conf set sqlagent.enabled true diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlserverCDC-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlserverCDC-source.md" new file mode 100644 index 0000000000..f69a5d3bf4 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/sqlservercdc/SqlserverCDC-source.md" @@ -0,0 +1,239 @@ +# SqlserverCDC Source + +## 一、介绍 + +Sqlservercdc插件支持配置监听表名称以及读取起点读取日志数据。SQLservercdc在checkpoint时保存当前消费的lsn,因此支持续跑。 + +## 二、支持版本 + +SqlServer 2012、2014、2016、2017、2019单机版 + +## 三、数据库配置 + +[SqlserverCDC配置](SqlServer配置CDC.md) + +## 四、SqlserverCDC原理 + +[SqlserverCDC原理](SqlServer CDC实时采集原理.md) + +## 五、插件名称 + +| Sync | sqlservercdcreader、sqlservercdcsource | +| --- | --- | +| SQL | sqlservercdc-x | + +## + +## 六、参数说明 + +### 1、Sync + +- **url** + - 描述:sqlserver数据库的JDBC URL链接 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **username** + - 描述:用户名 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **password** + - 描述:密码 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **tableList** + - 描述: 需要监听的表,如:["schema1.table1","schema1.table2"] + - 必选:是 + - 字段类型:数组 + - 默认值:无 + + +- **splitUpdate** + - 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见【六、数据结构说明】 + - 必选:否 + - 字段类型:boolean + - 默认值:false + + +- **cat** + - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 + - 必选:否 + - 字段类型:String + - 默认值:UPDATE,INSERT,DELETE + + +- **lsn** + - 描述: 要读取SqlServer CDC日志序列号的开始位置 + - 必选: 否 + - 字段类型:String(00000032:00002038:0005) + - 默认值:无 + + +- **pollInterval** + - 描述: 监听拉取SqlServer CDC数据库间隔时间,该值越小,采集延迟时间越小,给数据库的访问压力越大 + - 必选:否 + - 字段类型:long(单位毫秒) + - 默认值:1000 + + +- **pavingData** + - 描述:是否将解析出的json数据拍平,具体见【七、数据结构说明】 + - 必选:否 + - 字段类型:boolean + - 默认值:false + +### 2、SQL + +- **url** + - 描述:sqlserver数据库的JDBC URL链接 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **username** + - 描述:用户名 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **password** + - 描述:密码 + - 必选:是 + - 参数类型:string + - 默认值:无 + + +- **table** + - 描述:需要解析的数据表。 + - 注意:SQL任务只支持监听单张表,且数据格式为schema.table + - 必选:否 + - 字段类型:string + - 默认值:无 + + +- **cat** + - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 + - 必选:否 + - 字段类型:String + - 默认值:UPDATE,INSERT,DELETE + + +- **lsn** + - 描述: 要读取SqlServer CDC日志序列号的开始位置 + - 必选: 否 + - 字段类型:String(00000032:00002038:0005) + - 默认值:无 + + +- **poll-interval** + - 描述: 监听拉取SqlServer CDC数据库间隔时间,该值越小,采集延迟时间越小,给数据库的访问压力越大 + - 必选:否 + - 字段类型:long(单位毫秒) + - 默认值:1000 + +## 七、数据结构 + +在2020-01-01 12:30:00(时间戳:1577853000000)执行: + +```sql +INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) +VALUES (1, 1, 'a'); +``` + +在2020-01-01 12:31:00(时间戳:1577853060000)执行: + +```sql +DELETE +FROM `tudou`.`kudu` +WHERE `id` = 1 + AND `user_id` = 1 + AND `name` = 'a'; +``` + +在2020-01-01 12:32:00(时间戳:1577853180000)执行: + +```sql +UPDATE `tudou`.`kudu` +SET `id` = 2, + `user_id` = 2, + `name` = 'b' +WHERE `id` = 1 + AND `user_id` = 1 + AND `name` = 'a'; +``` + +1、pavingData = true, splitUpdate = false RowData中的数据依次为: + +``` +//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name +["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] +["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", 1, 1, "a", 2, 2, "b"] +``` + +2、pavingData = false, splitUpdate = false RowData中的数据依次为: + +``` +//schema, table, ts, opTime, type, before, after +["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] +["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE", {"id":1, "user_id":1, "name":"a"}, {"id":2, "user_id":2, "name":"b"}] +``` + +3、pavingData = true, splitUpdate = true RowData中的数据依次为: + +``` +//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name +["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, null, null, 1, 1, "a"] +["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", 1, 1, "a", null, null, null] + +//schema, table, ts, opTime, type, before_id, before_user_id, before_name +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", 1, 1, "a"] + +//schema, table, ts, opTime, type, after_id, after_user_id, after_name +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", 2, 2, "b"] +``` + +4、pavingData = false, splitUpdate = true RowData中的数据依次为: + +``` +//schema, table, ts, opTime, type, before, after +["tudou", "kudu", 6760525407742726144, 1577853000000, "INSERT", null, {"id":1, "user_id":1, "name":"a"}] +["tudou", "kudu", 6760525407742726144, 1577853060000, "DELETE", {"id":1, "user_id":1, "name":"a"}, null] +//schema, table, ts, opTime, type, before +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_BEFORE", {"id":1, "user_id":1, "name":"a"}] +//schema, table, ts, opTime, type, after +["tudou", "kudu", 6760525407742726144, 1577853180000, "UPDATE_AFTER", {"id":2, "user_id":2, "name":"b"}] +``` + +- type:变更类型,INSERT,UPDATE、DELETE +- opTime:数据库中SQL的执行时间 +- ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下: + +```java +long id=Long.parseLong("6760525407742726144"); + long res=id>>22; + DateFormat sdf=new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + System.out.println(sdf.format(res)); //2021-01-28 19:54:21 +``` + +## 八、数据类型 + +| 是否支持 | 数据类型 | +| --- | --- | +| 支持 | BIT, TINYINT24, INT, INTEGER, FLOAT, DOUBLE, REAL, LONG, BIGINT, DECIMAL, NUMERIC, BINARY, VARBINARY, DATE, TIME, TIMESTAMP, DATETIME, DATETIME2, SMALLDATETIME, CHAR, VARCHAR, NCHAR, NVARCHAR, TEXT | +| 不支持 | ROWVERSION, UNIQUEIDENTIFIER, CURSOR, TABLE, SQL_VARIANT | + +## 九、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\346\217\220\344\272\244\344\270\200\344\270\252\344\274\230\347\247\200\347\232\204PR.md" "b/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\346\217\220\344\272\244\344\270\200\344\270\252\344\274\230\347\247\200\347\232\204PR.md" new file mode 100644 index 0000000000..e3f17ca6ce --- /dev/null +++ "b/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\346\217\220\344\272\244\344\270\200\344\270\252\344\274\230\347\247\200\347\232\204PR.md" @@ -0,0 +1,100 @@ +# 如何提交一个优秀的pr +在github上提交pr是参与ChunJun开源项目的一个重要途径,小伙伴们在使用中的一些功能上feature或者bug都可以向社区提交pr贡献代码,也可以根据已有的issue提供自己的解决方案。下面给大家带来提交一个优秀PR的步骤。 + +## 第一步:fork chunjun到自己的github仓库 +![image](../../website/src/images/doc/pr/pr1.png) + +点击fork后就可以在自己仓库中看到以你名字命名的chunjun项目了: + +![image](../../website/src/images/doc/pr/pr2.png) + +## 第二步:clone chunjun到本地IDE +![image](../../website/src/images/doc/pr/pr3.png) + +## 第三步:将DTStack/chunjun设置为本地仓库的远程分支upstream +```shell +$ cd chunjun +# add upstream +$ git remote add upstream https://github.com/DTStack/chunjun.git +# 查看远程仓库设置 +$ git remote -v +origin https://github.com/your_name/chunjun.git (fetch) +origin https://github.com/your_name/chunjun.git (push) +upstream https://github.com/DTStack/chunjun.git (fetch) +upstream https://github.com/DTStack/chunjun.git (push) +``` +## 第四步:提交代码 +任何一个提交都要基于最新的分支 +**切换分支** +```shell +# Fetch branches from upstream. +$ git remote update upstream -p +# Checkout a new branch. +$ git checkout -b branch_name +# Pull latest code into your own branch. +$ git pull upstream master:branch_name +``` +**本地修改代码后,提交commit** +- commit message 规范: + [commit_type-#issue-id] [module] message +- commit_type: + - feat:表示是一个新功能(feature) + - hotfix:hotfix,修补bug + - docs:改动、增加文档 + - opt:修改代码风格及opt imports这些,不改动原有执行的代码 + - test:增加测试 +- eg:[hotfix-#12345][mysql] Fix mysql time type loses precision. + +注意: +(1)commit 需遵循规范,给维护者减少维护成本及工作量,对于不符合规范的commit,我们不与合并; +(2)对于解决同一个Issue的PR,只能存在一个commit message,如果出现多次提交的message,我们希望你能将commit message 'squash' 成一个; +(3)message 尽量保持清晰简洁,但是也千万不要因为过度追求简洁导致描述不清楚,如果有必要,我们也不介意message过长,前提是,能够把解决方案、修复内容描述清楚; + +```shell +# 提交commit前先进行代码格式化 +$ mvn spotless:apply +$ git commit -a -m "" +``` +**rebase远程分支** + +这一步很重要,因为我们仓库中的chunjun代码很有可能已经落后于社区,所以在 push commit前需要rebase,保证当前commit是基于社区最新的代码,很多小伙伴没有这一步导致提交的pr当中包含了其他人的commit +```shell +$ git fetch upstream +$ git rebase upstream/branch_name +``` + +*rebase后有可能出现代码冲突,一般是由于多人编辑同一个文件引起的,只需要根据提示打开冲突文件对冲突部分进行修改,将提示的冲突文件的冲突都解决后,执行 +```shell +$ git add . +$ git rebase --continue +``` +依此往复,直至屏幕出现类似rebase successful字样即可 + +*rebase之后代码可能无法正常推送,需要```git push -f``` 强制推送,强制推送是一个有风险的操作,操作前请仔细检查以避免出现无关代码被强制覆盖的问题 + +**push到github仓库** +```shell +$ git push origin branch_name +``` +## 第五步:提交pr +以笔者修复kafka写入过程中出现空指针问题为例,经过步骤四笔者已经把代码提交至笔者自己仓库的master分支 + +![image](../../website/src/images/doc/pr/pr4.png) + +进入chunjun仓库页面,点击Pull Request + +![image](../../website/src/images/doc/pr/pr5.png) + +![image](../../website/src/images/doc/pr/pr6.png) + +选择head仓库和base仓库以及相应的分支 + +![image](../../website/src/images/doc/pr/pr7.png) + +填写pr信息,pr信息应该尽量概括清楚问题的前因后果,如果存在对应issue要附加issue地址,保证问题是可追溯的 + +![image](../../website/src/images/doc/pr/pr8.png) + +![image](../../website/src/images/doc/pr/pr9.png) + +PR提交成功后需要一段时间代码review,可以耐心等待一下项目维护者review后合入,或者在PR评论区艾特相关人员。 diff --git "a/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\350\207\252\345\256\232\344\271\211\346\217\222\344\273\266.md" "b/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\350\207\252\345\256\232\344\271\211\346\217\222\344\273\266.md" new file mode 100644 index 0000000000..5537f3c5b8 --- /dev/null +++ "b/docs_zh/\345\274\200\345\217\221\350\200\205\346\214\207\345\215\227/\345\246\202\344\275\225\350\207\252\345\256\232\344\271\211\346\217\222\344\273\266.md" @@ -0,0 +1,337 @@ +## 如何自定义插件 + +本文面向ChunJun插件开发人员,尝试通过一个开发者的角度尽可能全面地阐述一个ChunJun插件所经历的过程,同时消除开发者的困惑,快速上手插件开发。 + +从数据流的角度来看ChunJun,可以理解为不同数据源的数据流通过对应的ChunJun插件处理,变成符合ChunJun数据规范的数据流;脏数据的处理可以理解为脏水流通过污水处理厂,变成符合标准,可以使用的水流,而对不能处理的水流收集起来。 + +插件开发不需要关注任务具体如何调度,只需要关注关键问题: + +1. 数据源本身读写数据的正确性; +2. 如何合理且正确地使用框架; +3. 配置文件的规范,每个插件都应有对应的配置文件; + +每个插件应当有以下目录: + +1. conf:存放插件配置类的包。 +2. converter:存放插件数据类型转换规则类的包。 +3. source:存放插件数据源读取逻辑有关类的包。 +4. sink:存放插件数据源写入逻辑有关类的包。 +5. table:存放插件数据源sql模式有关类的包。 +6. util:存放插件工具类的包,chunjun已经封装了一些常用工具类在chunjun-core模块中,如果还需编写插件工具类的请放在该插件目录中的util包。 + +### 一. Debug调试 + +#### (1)本地调试 + +在chunjun-local-test模块中,官方已经写好了本地测试的LocalTest类,只需更改脚本文件路径,在代码处打上断点即可调试。 + +![image-20220614171917692](../../website/src/images/doc/contribute/image-20220614171917692.png) + +#### (2)远程调试 + +如果需要远程调试,那么需要在 flink-conf.yaml 中增加 Flink 的远程调试配置,然后在 idea 中配置”JVM Remote“,在代码块中打断点(这种方法还能调试 Flink 本身的代码) + +```shell +env.java.opts.jobmanager: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005 + +env.java.opts.taskmanager: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006 +``` + +只需要修改标记的这两个地方,如果是 HA 集群,需要根据日志修改**怎么看日志,怎么修改,自行查资料**。 + +![image-20220614172338108](../../website/src/images/doc/contribute/image-20220614172338108.png) + +至此,任务 idea 调试流程就这些内容。 + +### 二. sync(json)插件 + +以Stream插件为例: + +#### (1) reader + +插件数据源读取逻辑需要继承BaseRichInputFormat类,BaseRichInputFormat是具体的输入数据的操作,包括open、nextRecord、close,每个插件具体操作自己的数据,InputFormat公共内容都在BaseRichInputFormat,不要随意修改。 + +创建StreamInputFormat类继承BaseRichInputFormat类,重写其中的必要方法。 + +```java +public class StreamInputFormat extends BaseRichInputFormat { + //创建数据分片 + @Override + public InputSplit[] createInputSplitsInternal(int minNumSplits) {......} + //打开数据连接 + @Override + public void openInternal(InputSplit inputSplit) {......} + //读取一条数据 + @Override + public RowData nextRecordInternal(RowData rowData) throws ReadRecordException {......} + //判断数据是否读取完毕 + @Override + public boolean reachedEnd() {......} + //关闭数据连接 + @Override + protected void closeInternal() {......} +} +``` + +StreamInputFormat类是通过StreamInputFormatBuilder类构建的。 + +```java +public class StreamInputFormatBuilder extends BaseRichInputFormatBuilder { + + private final StreamInputFormat format; + + public StreamInputFormatBuilder() { + super.format = format = new StreamInputFormat(); + } + //检查inputformat配置 + @Override + protected void checkFormat() {......} +} +``` + +创建StreamSourceFactory继承SourceFactory类 + +```java +public class StreamSourceFactory extends SourceFactory { + private final StreamConf streamConf; + + public StreamSourceFactory(SyncConf config, StreamExecutionEnvironment env) {......} + + //构建数据流读取对象 + @Override + public DataStream createSource() { + StreamInputFormatBuilder builder = new StreamInputFormatBuilder(); + builder.setStreamConf(streamConf); + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + rowConverter = new StreamColumnConverter(streamConf); + } else { + checkConstant(streamConf); + final RowType rowType = + TableUtil.createRowType(streamConf.getColumn(), getRawTypeConverter()); + rowConverter = new StreamRowConverter(rowType); + } + builder.setRowConverter(rowConverter, useAbstractBaseColumn); + + return createInput(builder.finish()); + } + //获取数据类型转换连接器,数据类型转换关系的实现 + @Override + public RawTypeConverter getRawTypeConverter() { + return StreamRawTypeConverter::apply; + } +} +``` + +StreamColumnConverter 继承 AbstractRowConverter类 是数据类型转换的具体实现,其中的方法参看源码。 + +接下来从作业执行角度阐述上述类之间的执行关系。 + +1. com.dtstack.chunjun.Main 入口类,通过判断启动参数来决定启动何种作业。 + + ![image-20220614143347037](../../website/src/images/doc/contribute/image-20220614143347037.png) + +2. 解析参数生成SyncConf对象,配置执行环境。 + +![image-20220614144650798](../../website/src/images/doc/contribute/image-20220614144650798.png) + +3. .将上面解析生成的SyncConf,然后通过反射加载具体的插件调用createSource方法生成DataStream + +![image-20220614145127874](../../website/src/images/doc/contribute/image-20220614145127874.png) + +4. createSource方法中会构建inputformat对象,然后调用createInput方法,将inputformat对象封装至DtInputFormatSourceFunction中。 + +![image-20220614145839069](../../website/src/images/doc/contribute/image-20220614145839069.png) + +5. DtInputFormatSourceFunction类中会调用inputformat对象中的逻辑去读取数据,inputformat中的nextRecordInternal方法读数据时,会对每条数据进行数据类型转换。 + +![image-20220614150742626](../../website/src/images/doc/contribute/image-20220614150742626.png) + +6. 数据类型转换,flink自己内部有一套自己的数据类型,用来和外部系统进行交互。交互过程分为:将外部系统数据按照定义的类型读入到flink内部、将外部数据转换成flink内部类型、将内部类型进行转换写到外部系统。所以每个插件需要有一套类型转换机制来满足数据交互的需求。 + +7. 将外部数据转换成flink内部类,每个插件的转换方式都不同。 + +![image-20220614151300210](../../website/src/images/doc/contribute/image-20220614151300210.png) + +#### (2) writer + +插件数据源读取逻辑需要继承BaseRichOutputFormat类,BaseRichOutputFormat是具体的输入数据的操作,包括open、writeRecord、close,每个插件具体操作自己的数据,OutputFormat公共内容都在BaseRichOutputFormat,不要随意修改。 + +创建StreamOutputformat类继承BaseRichOutputformat类,重写其中的必要方法。 + +```java +public class StreamOutputFormat extends BaseRichOutputFormat { + //打开资源 + @Override + protected void openInternal(int taskNumber, int numTasks) {......} + //写出单条数据 + @Override + protected void writeSingleRecordInternal(RowData rowData) throws WriteRecordException {......} + //写出多条数据 + @Override + protected void writeMultipleRecordsInternal() throws Exception {......} + //关闭资源 + @Override + protected void closeInternal() {......} +} +``` + +StreamOutputFormat类是通过StreamOutputFormatBuilder构建的 + +```java +public class StreamOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private StreamOutputFormat format; + + public StreamOutputFormatBuilder() { + super.format = format = new StreamOutputFormat(); + } + //检查Outputformat配置 + @Override + protected void checkFormat() {......} +} +``` + +创建StreamSinkFactory类继承SinkFactory类 + +```java +public class StreamSinkFactory extends SinkFactory { + + private final StreamConf streamConf; + + public StreamSinkFactory(SyncConf config) {......} + + //构建数据输出流对象 + @Override + public DataStreamSink createSink(DataStream dataSet) { + StreamOutputFormatBuilder builder = new StreamOutputFormatBuilder(); + builder.setStreamConf(streamConf); + AbstractRowConverter converter; + if (useAbstractBaseColumn) { + converter = new StreamColumnConverter(streamConf); + } else { + final RowType rowType = + TableUtil.createRowType(streamConf.getColumn(), getRawTypeConverter()); + converter = new StreamRowConverter(rowType); + } + + builder.setRowConverter(converter, useAbstractBaseColumn); + return createOutput(dataSet, builder.finish()); + } + //获取数据类型转换连接器,数据类型转换关系的实现 + @Override + public RawTypeConverter getRawTypeConverter() { + return StreamRawTypeConverter::apply; + } +} +``` + +接下来从作业执行角度阐述上述类之间的执行关系,在阐述reader插件执行步骤第三步时,通过反射加载具体的插件调用createSource方法生成DataStream,同理,在生成DataStream后,writer插件也是通过反射加载调用createSink方法生成DataStreamSink的。 + +![image-20220616102414067](../../website/src/images/doc/contribute/image-20220616102414067.png) + +1. createSink方法中会构建outputformat对象,然后调用createoutput方法,将outputformat对象封装至DtOutputFormatSinkFunction中。 + +![image-20220616102856402](../../website/src/images/doc/contribute/image-20220616102856402.png) + +2. DtOutputFormatSinkFunction类中会调用outputformat对象中的逻辑去写入数据,outputformat中的writeSingleRecordInternal方法写入数据时,会对每条数据进行数据类型转换。 + +![image-20220616103241458](../../website/src/images/doc/contribute/image-20220616103241458.png) + +### 三. sql插件 + +**Flink SQL Connetor 详细设计文档参见**[FLIP-95](https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces) + +Flink SQL Connector的架构简图如下所示: + +195230-11f4ee6bc7e788c7.webp + +动态表一直都是Flink SQL流批一体化的重要概念,也是上述架构中Planning阶段的核心。而自定义Connector的主要工作就是实现基于动态表的Source/Sink,还包括上游产生它的工厂,以及下游在Runtime阶段实际执行Source/Sink逻辑的RuntimeProvider。 + +DynamicTableFactory需要具备以下功能: + +- 定义与校验建表时传入的各项参数; +- 获取表的元数据; +- 定义读写数据时的编码/解码格式(非必需); +- 创建可用的DynamicTable[Source/Sink]实例。 + +DynamicTableSourceFactory:源表工厂,里面包含从source来的表(如:源表kafka),从lookup来的表(如:维表mysql) + +DynamicTableSinkFactory:结果表工厂,里面包含从sink来的表(如:结果表mysql) + +DynamicTableSource:生成需要读取数据的RichSourceFunction(面向流)\RichInputFormat(面向批,也可以用于流),实现ScanTableSource即可得到。生成需要读取数据的TableFunction(全量)\AsyncTableFunction(lru),实现LookupTableSource即可。并被包装成Provider。 + +DynamicTableSink:生成需要写出数据的RichSinkFunction(面向流)\RichOutputFormat(面向批,也可以用于流)。并被包装成Provider。 + +如果一个插件需要source端包含源表和维表,则实现ScanTableSource和LookupTableSource接口 + +如果一个插件需要sink端,则实现DynamicTableSink接口 + +实现了DynamicTable[Source/Sink]Factory接口的工厂类如下所示。 + +```java +public class StreamDynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { + //创建DynamicTableSource + @Override + public DynamicTableSource createDynamicTableSource(Context context) { } + //创建DynamicTableSink + @Override + public DynamicTableSink createDynamicTableSink(Context context) { } + //connector唯一标识符 + @Override + public String factoryIdentifier() { } + //必选参数设置 + @Override + public Set> requiredOptions() { } + //可选参数设置 + @Override + public Set> optionalOptions() { } +} +``` + +1. 根据Connector特性是否只用到source、sink,实现对应的接口,这里以stream为例:既可以作为源表、又可以作为结果表,所以实现如下: + +![image-20220614152751238](../../website/src/images/doc/contribute/image-20220614152751238.png) + +2. 实现createDynamicTableSource方法用来创建DynamicTableSource(ScanTableSource),在创建之前,我们可以利用内置的TableFactoryHelper工具类来校验传入的参数,当然也可以自己编写校验逻辑。另外,通过关联的上下文对象还能获取到表的元数据。 + +![image-20220614152928836](../../website/src/images/doc/contribute/image-20220614152928836.png) + +3. 创建StreamDynamicTableSource类,目前stream实现了ScanTableSource,实现getScanRuntimeProvider方法,用来创建DtInputFormatSourceFunction(并创建StreamInputFormat包含在DtInputFormatSourceFunction中) + + DtInputFormatSourceFunction:是所有InputFormat的包装类,里面实现类2pc等功能,不要随意修改。 + +![image-20220614153545215](../../website/src/images/doc/contribute/image-20220614153545215.png) + +4. 最后创建StreamInputFormat,用来对数据的操作包括(open、writeRecord、close方法),公共的内容已经抽取到了BaseRichInputFormat中,是所有OutputFormat的公共类,不要随意修改。 + +5. sink类似。 + +6. 插件也可按需求实现LookupTableSource类,以jdbc插件为例,实现getLookupRuntimeProvider方法,创建JdbcLruTableFunction/JdbcAllTableFunction, + + ```tex + 维表,支持全量和异步方式 + 全量缓存:将维表数据全部加载到内存中,建议数据量大不使用。 + 异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 + ``` +![images](../../website/src/images/doc/contribute/image-20220616112115980.png) + +7. JdbcLruTableFunction继承AbstractLruTableFunction,重写其必要方法,JdbcAllTableFunction继承AbstractAllTableFunction,具体逻辑实现参考源码。 + +8. Flink SQL采用SPI机制来发现与加载表工厂类。所以最后不要忘了classpath的META-INF/services目录下创建一个名为`org.apache.flink.table.factories.Factory`的文件,并写入我们自定义的工厂类的全限定名,如:com.dtstack.chunjun.connector.stream.table.StreamDynamicTableFactory + +### 四. 插件打包 + +进入项目根目录,使用maven打包,有关打包配置请参考其他插件的pom文件。 + +```sh +mvn clean package -DskipTests +``` + +打包之前注意代码格式,在项目根目录执行以下命令格式化代码。 + +```shell +mvn spotless:apply +``` + +打包结束后,项目根目录下会产生chunjun-dist目录,如果没意外的话,您开发的插件会在connetor目录下,之后就可以提交开发平台测试啦! diff --git "a/docs_zh/\345\277\253\351\200\237\345\274\200\345\247\213.md" "b/docs_zh/\345\277\253\351\200\237\345\274\200\345\247\213.md" new file mode 100644 index 0000000000..670340a336 --- /dev/null +++ "b/docs_zh/\345\277\253\351\200\237\345\274\200\345\247\213.md" @@ -0,0 +1,205 @@ +# 快速开始 + +本文讨论如何在不同场景下使用纯钧进行同步/SQL计算任务。本文中以 Stream -> Stream 为例子说明,如需构建其他数据源任务,请根据插件文档具体修改。 + +# 准备开始 + +操作系统:无限制 + +系统版本:无限制 + +### 环境要求 + +#### java + +- JDK1.8 +- 配置好JAVA_HOME环境变量 + +> jdk安装步骤这里不做介绍 + +#### Flink + +- 推荐flink1.12.7 +- 官网下载压缩包直接解压即可 + - 下载地址:https://dlcdn.apache.org/flink/flink-1.12.7/flink-1.12.7-bin-scala_2.12.tgz +- 需要在提交机器上配置FLINK_HOME环境变量 +- yarn-session和yarn-perJob任务需要额外在$FLINK_HOME/lib/目录下加入flink-shaded-hadoop-2-uber-2.7.5-10.0.jar文件 + - 下载地址:https://repo1.maven.org/maven2/org/apache/flink/flink-shaded-hadoop-2/2.7.5-10.0/flink-shaded-hadoop-2-2.7.5-10.0.jar + +> local模式不需要安装Flink + +#### Hadoop + +- 推荐hadoop2.7.5 +- 需要在提交机器上配置HADOOP_HOME环境变量 + +> local模式和standalone模式无需依赖Hadoop环境 + +# 获取插件 + +纯钧提供了已经编译好的插件压缩包([chunjun-dist.tar](https://github.com/DTStack/chunjun/releases)),里面包含目前所有的脚本案例,任务提交脚本,插件包等内容,使得用户可以直接下载,根据需要配置任务,开箱即用。 + +另外,可以下载源码([github地址](https://github.com/DTStack/chunjun)),自行编译源码,提交任务。 + +## 压缩包 + +纯钧提供的压缩包(chunjun-dist.tar)里包含四部分内容:bin(包含任务提交脚本),chunjun-dist(纯钧任务插件包),chunjun-example(纯钧任务脚本模版),lib(任务提交客户端),用户可以通过bin里的提交脚本,使用已经编译好的插件jar包直接提交任务,无需关心插件编译过程,适合调研使用。 + +## 源码编译 + +### 获取代码 +1.使用git工具把项目clone到本地 + +``` +git clone https://github.com/DTStack/chunjun.git +cd chunjun +``` + +### 插件编译 +在chunjun目录下执行 + +```bash +mvn clean package -DskipTests +``` + +或者执行 + +```bash +sh build/build.sh +``` + +执行完上述命令之后,在chunjun-assembly模块的target目录会得到一个完整的安装包 + +### 多平台兼容 + +chunjun目前支持tdh和开源hadoop平台,对不同的平台有需要使用不同的maven命令打包 + +| 平台类型 | | 含义 | +| -------- | -------------------------------------------- | --------------------------------------- | +| tdh | mvn clean package -DskipTests -P default,tdh | 打包出inceptor插件以及default支持的插件 | +| default | mvn clean package -DskipTests -P default | 除了inceptor插件之外的所有插件 | + +### 常见问题 + +#### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 + +解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: + +```bash +## windows平台 +./$CHUNJUN_HOME/bin/install_jars.bat + +## unix平台 +./$CHUNJUN_HOME/bin/install_jars.sh +``` + +# 任务提交 + +纯钧支持多种模式提交任务,在生产环境中,常用的模式有yarn-session和 yarn-pre-job 模式。 +在开始前请先确保环境安装正确。 + +视频不演示资源下载过程以及jdk、hadoop环境的安装过程 + +[视频链接](https://www.bilibili.com/video/BV1ZS4y1H74k?spm_id_from=333.999.0.0) + +## 参数说明 + +mode:任务提交的类型,非必填项,类型有:local(默认值),standalone,yarn-session,yarn-per-job,kubernetes-session,kubernetes-application,对应源码中枚举类 **ClusterMode**; + +jobType:纯钧任务类型,必填项,同步任务为:sync,SQL计算任务为:sql; + +job:纯钧任务脚本地址,必填项; + +chunjunDistDir:纯钧插件包地址; + +confProp:纯钧任务配置参数,Flink相关配置也是在这里配置; + +flinkConfDir:flink-conf.yaml 地址,在非local模式时,需要配置; + +## Local + +Local 模式不依赖Flink环境和Hadoop环境,在本地环境启动一个JVM进程执行纯钧任务。 + +### 提交步骤 + +进入到chunjun-dist 目录,执行命令 + +```shell +sh bin/chunjun-local.sh -job chunjun-examples/json/stream/stream.json +``` + +即可执行一个简单的 **stream -> stream** 同步任务 + +[参考视频](https://www.bilibili.com/video/BV1mT411g7fJ?spm_id_from=333.999.0.0) + +## Standalone + +Standalone模式依赖Flink Standalone环境,不依赖Hadoop环境。 + +### 提交步骤 + +#### 1. 启动Flink Standalone环境 + +```shell +sh $FLINK_HOME/bin/statrt-cluster.sh +``` + +启动成功后默认端口为8081,我们可以访问当前机器的8081端口进入standalone的flink web ui + +#### 2. 提交任务 + +进入到本地chunjun-dist目录,执行命令 + +```shell +sh bin/chunjun-standalone.sh -job chunjun-examples/json/stream/stream.json +``` + +提交成功之后,可以在flink web ui 上观察任务情况; + +[参考视频](https://www.bilibili.com/video/BV1TT41137UV?spm_id_from=333.999.0.0) + +## Yarn Session + +YarnSession 模式依赖Flink 和 Hadoop 环境,需要在任务提交之前启动相应的yarn session; + +### 提交步骤 + +#### 1. 启动yarn session环境 + +Yarn Pre-Job 模式依赖Flink 和 Hadoop 环境,需要在提交机器中提前设置好$HADOOP_HOME和$FLINK_HOME + +我们需要使用yarn-session -t参数上传chunjun-dist + +```shell +cd $FLINK_HOME/bin +./yarn-session -t $CHUNJUN_HOME -d +``` + +#### 2. 提交任务 + +通过yarn web ui 查看session 对应的application $SESSION_APPLICATION_ID,进入到本地chunjun-dist目录,执行命令 + +```shell +sh ./bin/chunjun-yarn-session.sh -job chunjun-examples/json/stream/stream.json -confProp {\"yarn.application.id\":\"SESSION_APPLICATION_ID\"} +``` + +yarn.application.id 也可以在 flink-conf.yaml 中设置;提交成功之后,可以通过 yarn web ui 上观察任务情况。 + +[参考视频](https://www.bilibili.com/video/BV1oU4y1D7e7?spm_id_from=333.999.0.0) + +## Yarn Per-Job + +Yarn Per-Job 模式依赖Flink 和 Hadoop 环境,需要在提交机器中提前设置好$HADOOP_HOME和$FLINK_HOME。 + +### 提交步骤 + +Yarn Per-Job 提交任务配置正确即可提交。进入本地chunjun-dist目录,执行命令提交任务。 + +```shell +sh ./bin/chunjun-yarn-perjob.sh -job chunjun-examples/json/stream/stream.json +``` + +提交成功之后,可以通过 yarn web ui 上观察任务情况; + +[参考视频](https://www.bilibili.com/video/BV1oU4y1D7e7?spm_id_from=333.999.0.0) + diff --git "a/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\345\242\236\351\207\217\345\220\214\346\255\245\344\273\213\347\273\215.md" "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\345\242\236\351\207\217\345\220\214\346\255\245\344\273\213\347\273\215.md" new file mode 100644 index 0000000000..2f6e208e99 --- /dev/null +++ "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\345\242\236\351\207\217\345\220\214\346\255\245\344\273\213\347\273\215.md" @@ -0,0 +1,130 @@ +# 什么是增量同步 + +​ 增量同步主要针对某些只有Insert操作的表,随着业务增长,表内数据越来越多。如果每次都同步整表的话,消耗的时间和资源会比较多。因此需要一个增量同步的功能,每次只读取增加部分的数据。 + +## 原理解析 + +​ 其实现原理实际上就是配合增量键在查询的sql语句中拼接过滤条件,比如 where id > ? ,将之前已经读取过的数据过滤出去。 + +​ 增量同步是针对于两个及以上的同步作业来说的。对于初次执行增量同步的作业而言,实际上是整表同步,不同于其他作业的在于增量同步作业会在作业执行完成后记录一个endLocation指标,并将这个指标上传到prometheus以供后续使用。除第一次作业外,后续的所有增量同步作业都会取上一次作业的endLocation做为本次作业的过滤依据(startLocation)。比如第一次作业执行完后,endLocation为10,那么下一个作业就会构建出例如 SELECT id,name,age from table where id > 10的SQL语句,达到增量读取的目的。 + +> 增量键:数据库表中增量递增的字段,比如自增id + +## 使用限制 + +- 只有RDB的Reader插件插件可以使用 + - 通过构建SQL过滤语句实现,因此只能用于RDB插件 + - 增量同步只关心读,不关心写,因此只与reader插件有关 +- 增量字段只能为数值类型和时间类型 + - 指标需要上传到prometheus,而prometheus不支持字符串类型,因此只支持数据类型和时间类型。时间类型会转换成时间戳后上传 +- 增量键的值可以重复,但必须递增 + - 由于使用'>'的缘故,要求字段必须递增。 + +## 如何处理增量键重复场景 + +​ 考虑可能存在这样的场景:某一次增量同步后的endLocation为x,在下一次增量同步作业启动的间隙中,表内又写入了增量键的值=x的数据。按照默认的情况,假设增量键为id,下一次作业会拼接例如SELECT id,name,age FROM table WHERE id > x。此时在间隙中插入的id=x的数据将会丢失。 + +​ 为了对应上述场景,chunjun增量同步提供了配置项useMaxFunc(默认值为false)。在设置useMaxFunc=true时,chunjun会在增量作业启动时获取当前数据库中增量键的最大值作为本次作业的endLocation,并且将用于startLocation的运算符号从'>'改为'>='。例如: + +- 某一次增量启动时上次作业的endLocation为10,id最大值为100,那么将会拼接SQL语句 SELECT id,name,age FROM table WHERE id >= 10 AND id < 100 +- 下一次增量作业启动时id的最大值为200,那么将会拼接SQL语句 SELECT id,name,age FROM table WHERE id >=100 AND id < 200 + + + +# 如何使用增量同步 + +## 环境准备 + +- 由于是使用prometheus收集相关指标信息,因此需要先安装prometheus和pushgateway。 + +- 下载Flink Metric Prometheus依赖,将其放入Flink lib目录下 + + - https://repo1.maven.org/maven2/org/apache/flink/flink-metrics-prometheus_2.12/1.12.7/flink-metrics-prometheus_2.12-1.12.7.jar + +- 修改Flink配置文件,conf/flink-conf.yaml,新增flink metric相关配置 + + ```yaml + metrics.reporter.promgateway.host: host01 + metrics.reporter.promgateway.port: 9091 + metrics.reporter.promgateway.deleteOnShutdown: false + metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter + ``` + +## 脚本配置 + +主要的配置项就是increColumn和startLocation + +以mysql为例脚本如下: + +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "mysqlreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "age", + "type": "int" + } + ], + "customSql": "", + "increColumn": "id", //指定增量字段进行增量同步,增量字段必须是column存在的字段 + "startLocation": "2", //第一次执行时为空,可配置字符串或不配置,后续提交的作业使用prometheus中指标值 + "username": "root", + "password": "root", + "connection": [ + { + "jdbcUrl": [ + "jdbc:mysql://localhost:3306/test?useSSL=false" + ], + "table": [ + "baserow" + ] + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": false + } + } + } + ], + "setting": { + "restore": { + "restoreColumnName": "id" + }, + "speed": { + "channel": 1, + "bytes": 0 + } + } + } +} + +``` + + + +## 查询prometheus + +使用flink作业对应的JobId在普罗米修斯中查询endLocation指标值 + +``` +flink_taskmanager_job_task_operator_flinkx_endlocation{job_id="xxx"} +``` + +![image-20220508231718458](../../website/src/images/doc/incremental/prometheus-search.png) diff --git "a/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\346\226\255\347\202\271\347\273\255\344\274\240\344\273\213\347\273\215.md" "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\346\226\255\347\202\271\347\273\255\344\274\240\344\273\213\347\273\215.md" new file mode 100644 index 0000000000..6fc47e617d --- /dev/null +++ "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\346\226\255\347\202\271\347\273\255\344\274\240\344\273\213\347\273\215.md" @@ -0,0 +1,29 @@ +# 断点续传介绍 +断点续传是为了在离线同步的时候,针对长时间同步任务如超过1天,如果在同步过程中由于某些原因导致任务失败,从头再来的话成本非常大,因此需要一个断点续传的功能从任务失败的地方继续。 +## 原理 + +- 基于flink的checkpoint,在checkpoint的时候 会存储source端最后一条数据的某个字段值,sink端插件执行事务提交。 +- 在任务失败,后续通过checkpoint重新运行时,source端在生成select语句的时候将state里的值作为条件拼接进行数据的过滤,达到从上次失败位点进行恢复 + +![image](../../website/src/images/doc/restore/restore1.png) + +jdbcInputFormat在拼接读取sql时,如果从checkpoint恢复的state不为空且restoreColumn不为空,则此时会将checkpoint里的state作为起点开始读取数据 +# 适用场景 +通过上述原理我们可知道 source端必须是RDB类型插件,因为是通过select语句拼接where条件进行数据过滤达到断点续传的,同时断点续传需要指定一个字段作为过滤条件,且此字段要求是递增的 + +- 任务需要开启checkpoint +- reader为RDB的插件均支持且writer支持事务的插件(如 rdb filesystem等),如果下游是幂等性则writer插件也不需要支持事务 +- 作为断点续传的字段在源表里的数据是递增的,因为过滤条件是 > +## 参数配置 +开启restore需要在脚本的restore里配置相关参数 + +| 参数 | 含义 | 类型 | +| --- | --- | --- | +| isResore | 是否开启断点续传,true代表开启断点续传,默认不开启 | boolean | +| restoreColumnName | 断点续传字段 | string | +| restoreColumnIndex | 断点续传字段在reader里的column的位置 | int | + +### 示例 +![image](../../website/src/images/doc/restore/restore2.png) + + diff --git "a/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\350\204\217\346\225\260\346\215\256\346\217\222\344\273\266\350\256\276\350\256\241.md" "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\350\204\217\346\225\260\346\215\256\346\217\222\344\273\266\350\256\276\350\256\241.md" new file mode 100644 index 0000000000..5bcf294856 --- /dev/null +++ "b/docs_zh/\346\213\223\345\261\225\345\212\237\350\203\275/\350\204\217\346\225\260\346\215\256\346\217\222\344\273\266\350\256\276\350\256\241.md" @@ -0,0 +1,96 @@ +--- +title: 脏数据插件 +sidebar_position: 2 +--- +## 背景 +目前任务对脏数据的处理仅仅是日志打印,这样显然是无法应对客户多变的使用场景。 +## 方案 +整体架构采用生产者-消费者模式,任务启动过程中,同时将Manager初始化并启动Consumer异步线程池,仅需在BaseRichInputFormat 和 BaseRichOutputFormat 调用 Manager的collect() 方法收集脏数据即可。 +流程图 +![image](../../website/src/images/doc/dirty/dirty-1.png) + +![image](../../website/src/images/doc/dirty/dirty-2.png) + +## 详细描述 +任务配置参数 +对应Java 实体类 - DirtyConf +![image](../../website/src/images/doc/dirty/dirty-conf.png) + +* type + 插件类型,必填项,根据type动态加载对应的插件; +* printRate + 脏数据在日志中的打印频率,默认值修改为1,表示默认脏数据信息都会打印到日志文件中,同时,如果printRate <= 0,表示不打印任何脏数据信息; +* errorLimit + 脏数据在插件中,处理失败的条数限制,当处理失败的脏数据条数超过这个限制时,任务抛出NoRestartException,即任务失败且不重试;默认值修改为1,如果errorLimit < 0,表示任务容忍所有的异常,不失败; +* totalLimit + 脏数据总条数限制,即收集到的脏数据超过这个限制时,任务抛出NoRestartException,即任务失败且不重试;默认值修改为1,如果totalLimit < 0,表示任务容忍所有的异常,不失败; +* properties + 各自插件的参数配置 + +脏数据插件管理者 +对应Java 实体类 - DirtyManager +![image](../../website/src/images/doc/dirty/dirty-manager.png) +![image](../../website/src/images/doc/dirty/dirty-manager.png) + +Manager主要维护着脏数据消费者Consumer 和 一个异步线程池; +* 主要作用是收集脏数据,并下发到Consumer队列中 + +* 调用 collect() 方法 + BaseRichInputFormat + + +脏数据插件消费者 +对应Java 实体类 - AbstractDirtyConsumer +Consumer主要维护着一个消息队列,中间缓存着脏数据信息; +* run() 方法 + 主要逻辑是消费队列中的脏数据,consume() 方法交给子类去实现;如果在consume过程中出现了异常,那么errorCounter 计数加一。 + +* consume() 方法 + 处理脏数据的具体逻辑,交由子类实现,根据插件的不同,对脏数据处理逻辑也会有所不同。 + + +## 以下参数在ChunJun 启动参数-confProp中 +```yaml +chunjun.dirty-data.output-type = log/jdbc +chunjun.dirty-data.max-rows = 1000 // total limit +chunjun.dirty-data.max-collect-failed-rows = 1000 // error limit + +chunjun.dirty-data.jdbc.url= +chunjun.dirty-data.jdbc.username= +chunjun.dirty-data.jdbc.password= +chunjun.dirty-data.jdbc.database= // database 可以写在 url +chunjun.dirty-data.jdbc.table= + +chunjun.dirty-data.log.print-interval= 500 +``` +JDBC 建表语句(MySQL) + +```sql +CREATE TABLE IF NOT EXISTS chunjun_dirty_data +( +job_id VARCHAR(32) NOT NULL COMMENT 'Flink Job Id', +job_name VARCHAR(255) NOT NULL COMMENT 'Flink Job Name', +operator_name VARCHAR(255) NOT NULL COMMENT '出现异常数据的算子名,包含表名', +dirty_data TEXT NOT NULL COMMENT '脏数据的异常数据', +error_message TEXT COMMENT '脏数据中异常原因', +field_name VARCHAR(255) COMMENT '脏数据中异常字段名', +create_time TIMESTAMP(6) DEFAULT CURRENT_TIMESTAMP(6) NOT NULL ON UPDATE CURRENT_TIMESTAMP(6) COMMENT '脏数据出现的时间点' +) +COMMENT '存储脏数据'; + +CREATE INDEX idx_job_id ON chunjun_dirty_data (job_id); +CREATE INDEX idx_operator_name ON chunjun_dirty_data(operator_name); +CREATE INDEX idx_create_time ON chunjun_dirty_data(create_time); +Metircs +chunjun_DirtyData_count +chunjun_DirtyData_collectFailedCount + +``` + +## 项目目录结构 +### 父模块 +chunjun-dirty + +### 子模块 +chunjun-dirty-mysql +chunjun-dirty-log diff --git a/website/.prettierignore b/website/.prettierignore new file mode 100644 index 0000000000..58d06c368a --- /dev/null +++ b/website/.prettierignore @@ -0,0 +1,4 @@ +.cache +package.json +package-lock.json +public diff --git a/website/.prettierrc b/website/.prettierrc new file mode 100644 index 0000000000..33d2cfa3f6 --- /dev/null +++ b/website/.prettierrc @@ -0,0 +1,4 @@ +{ + "arrowParens": "avoid", + "semi": false +} diff --git a/website/LICENSE b/website/LICENSE new file mode 100644 index 0000000000..7e964c1ee5 --- /dev/null +++ b/website/LICENSE @@ -0,0 +1,14 @@ +The BSD Zero Clause License (0BSD) + +Copyright (c) 2020 Gatsby Inc. + +Permission to use, copy, modify, and/or distribute this software for any +purpose with or without fee is hereby granted. + +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. diff --git a/website/gatsby-browser.js b/website/gatsby-browser.js new file mode 100644 index 0000000000..40e540b7fe --- /dev/null +++ b/website/gatsby-browser.js @@ -0,0 +1,49 @@ +/** + * Implement Gatsby's Browser APIs in this file. + * + * See: https://www.gatsbyjs.com/docs/browser-apis/ + */ +require("./src/styles/global.css") +const React = require("react") +// const { default: AppHeader } = require("./src/components/AppHeader") + +const Layout = require("./src/components/documentsMenu/menu").default +const JsonLayout = require("./src/components/jsonMenu/menu").default +const SqlLayout = require("./src/components/sqlMenu/menu").default +const SpaceLayout = require("./src/components/space/spaceLayout").default +// const AppFooter = require("./src/components/AppFooter").default + +// You can delete this file if you're not using it +// import("./src/assets/sass/index.scss") +exports.wrapPageElement = ({ element, props }) => { + // props provide same data to Layout as Page element will get + // including location, data, etc - you don't need to pass it + + const flag = element.key.includes("documents") || element.key.includes("examples") || element.key.includes("download") + console.log(element.key) + return ( +
+ {flag ? ( + <> + {element.key.includes("documents") && {element}} + {element.key.includes("examples/json") && {element}} + {element.key.includes("examples/sql") && {element}} + + ) : ( + {element} + )} +
+ ) +} + +exports.shouldUpdateScroll = ({ routerProps: { location } }) => { + const { pathname } = location + // list of routes for the scroll-to-top-hook + const scrollToTopRoutes = [`/documents`, `/examples`] + // if the new route is part of the list above, scroll to top (0, 0) + if (scrollToTopRoutes.find(path => pathname.includes(path))) { + document.querySelector(".main")?.scrollTo(0, 0) + } + + return false +} diff --git a/website/gatsby-config.js b/website/gatsby-config.js new file mode 100644 index 0000000000..87e7310fe3 --- /dev/null +++ b/website/gatsby-config.js @@ -0,0 +1,84 @@ +const path = require("path") +module.exports = { + siteMetadata: { + title: `Gatsby Default Starter`, + description: `Kick off your next, great Gatsby project with this default starter. This barebones starter ships with the main Gatsby configuration files you might need.`, + author: `@gatsbyjs`, + siteUrl: `https://gatsbystarterdefaultsource.gatsbyjs.io/`, + }, + plugins: [ + `gatsby-plugin-postcss`, + `gatsby-plugin-react-helmet`, + `gatsby-plugin-image`, + `gatsby-plugin-sass`, + "gatsby-plugin-mantine", + //中文文档源 + { + resolve: `gatsby-source-filesystem`, + options: { + name: `docs`, + path: path.resolve(__dirname, "../docs_zh"), + }, + }, + { + resolve: `gatsby-source-filesystem`, + options: { + name: `examples`, + path: path.resolve(__dirname, "../chunjun-examples"), + }, + }, + { + resolve: `gatsby-source-filesystem`, + options: { + name: `images`, + path: path.resolve(__dirname, "src/images"), + }, + }, + + `gatsby-transformer-sharp`, + `gatsby-plugin-sharp`, + `json-content`, + // `gatsby-transformer-plaintext`, + { + resolve: `gatsby-transformer-remark`, + options: { + plugins: [ + { + resolve: `gatsby-remark-autolink-headers`, + options: { + offsetY: `100`, + icon: ``, + className: `custom-class`, + maintainCase: true, + removeAccents: true, + isIconAfterHeader: true, + elements: [`h1`, `h2`], + }, + }, + { + resolve: `gatsby-remark-images`, + options: {}, + }, + ], + }, + }, + + { + resolve: `gatsby-plugin-manifest`, + options: { + name: `gatsby-starter-default`, + short_name: `starter`, + start_url: `/`, + background_color: `#663399`, + // This will impact how browsers show your PWA/website + // https://css-tricks.com/meta-theme-color-and-trickery/ + // theme_color: `#663399`, + display: `minimal-ui`, + icon: `src/images/logo-dark.svg`, // This path is relative to the root of the site. + }, + }, + // this (optional) plugin enables Progressive Web App + Offline functionality + // To learn more, visit: https://gatsby.dev/offline + // `gatsby-plugin-offline`, + ], +} diff --git a/website/gatsby-node.js b/website/gatsby-node.js new file mode 100644 index 0000000000..b3e9dd8cca --- /dev/null +++ b/website/gatsby-node.js @@ -0,0 +1,9 @@ +exports.createPages = async ({ actions }) => { + const { createPage } = actions + createPage({ + path: "/using-dsg", + component: require.resolve("./src/templates/using-dsg.js"), + context: {}, + defer: true, + }) +} diff --git a/website/gatsby-ssr.js b/website/gatsby-ssr.js new file mode 100644 index 0000000000..01d83255f4 --- /dev/null +++ b/website/gatsby-ssr.js @@ -0,0 +1,7 @@ +/** + * Implement Gatsby's SSR (Server Side Rendering) APIs in this file. + * + * See: https://www.gatsbyjs.com/docs/ssr-apis/ + */ + +// You can delete this file if you're not using it diff --git a/website/package-lock.json b/website/package-lock.json new file mode 100644 index 0000000000..f1ca8f3276 --- /dev/null +++ b/website/package-lock.json @@ -0,0 +1,36381 @@ +{ + "name": "gatsby-starter-default", + "version": "0.1.0", + "lockfileVersion": 2, + "requires": true, + "packages": { + "": { + "name": "gatsby-starter-default", + "version": "0.1.0", + "license": "0BSD", + "dependencies": { + "@icon-park/react": "^1.3.5", + "@mantine/core": "^4.2.5", + "@mantine/hooks": "^4.2.5", + "gatsby": "^4.14.0", + "gatsby-plugin-gatsby-cloud": "^4.14.0", + "gatsby-plugin-image": "^2.14.1", + "gatsby-plugin-manifest": "^4.14.0", + "gatsby-plugin-mantine": "^4.0.0", + "gatsby-plugin-offline": "^5.14.0", + "gatsby-plugin-react-helmet": "^5.14.0", + "gatsby-plugin-sass": "^5.14.0", + "gatsby-plugin-sharp": "^4.14.1", + "gatsby-source-filesystem": "^4.14.0", + "gatsby-transformer-remark": "^5.14.0", + "gatsby-transformer-sharp": "^4.14.0", + "node-sass": "^6.0.1", + "prop-types": "^15.8.1", + "react": "^17.0.1", + "react-dom": "^17.0.1", + "react-helmet": "^6.1.0" + }, + "devDependencies": { + "prettier": "^2.6.2" + } + }, + "node_modules/@ampproject/remapping": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.2.0.tgz", + "integrity": "sha512-qRmjj8nj9qmLTQXXmaR1cck3UXSRMPrbsLJAasZpF+t3riI71BXed5ebIOYwQntykeZuhjsdweEc9BxH5Jc26w==", + "dependencies": { + "@jridgewell/gen-mapping": "^0.1.0", + "@jridgewell/trace-mapping": "^0.3.9" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@ardatan/aggregate-error": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/@ardatan/aggregate-error/-/aggregate-error-0.0.6.tgz", + "integrity": "sha512-vyrkEHG1jrukmzTPtyWB4NLPauUw5bQeg4uhn8f+1SSynmrOcyvlb1GKQjjgoBzElLdfXCRYX8UnBlhklOHYRQ==", + "dependencies": { + "tslib": "~2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@ardatan/aggregate-error/node_modules/tslib": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.0.3.tgz", + "integrity": "sha512-uZtkfKblCEQtZKBF6EBXVZeQNl82yqtDQdv+eck8u7tdPxjLu2/lp5/uPW+um2tpuxINHWy3GhiccY7QgEaVHQ==" + }, + "node_modules/@babel/code-frame": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.16.7.tgz", + "integrity": "sha512-iAXqUn8IIeBTNd72xsFlgaXHkMBMt6y4HJp1tIaK465CWLT/fG1aqB7ykr95gHHmlBdGbFeWWfyB4NJJ0nmeIg==", + "dependencies": { + "@babel/highlight": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/compat-data": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.17.10.tgz", + "integrity": "sha512-GZt/TCsG70Ms19gfZO1tM4CVnXsPgEPBCpJu+Qz3L0LUDsY5nZqFZglIoPC1kIYOtNBZlrnFT+klg12vFGZXrw==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/core": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.17.10.tgz", + "integrity": "sha512-liKoppandF3ZcBnIYFjfSDHZLKdLHGJRkoWtG8zQyGJBQfIYobpnVGI5+pLBNtS6psFLDzyq8+h5HiVljW9PNA==", + "dependencies": { + "@ampproject/remapping": "^2.1.0", + "@babel/code-frame": "^7.16.7", + "@babel/generator": "^7.17.10", + "@babel/helper-compilation-targets": "^7.17.10", + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helpers": "^7.17.9", + "@babel/parser": "^7.17.10", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.10", + "@babel/types": "^7.17.10", + "convert-source-map": "^1.7.0", + "debug": "^4.1.0", + "gensync": "^1.0.0-beta.2", + "json5": "^2.2.1", + "semver": "^6.3.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/core/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@babel/core/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/eslint-parser": { + "version": "7.17.0", + "resolved": "https://registry.npmjs.org/@babel/eslint-parser/-/eslint-parser-7.17.0.tgz", + "integrity": "sha512-PUEJ7ZBXbRkbq3qqM/jZ2nIuakUBqCYc7Qf52Lj7dlZ6zERnqisdHioL0l4wwQZnmskMeasqUNzLBFKs3nylXA==", + "dependencies": { + "eslint-scope": "^5.1.1", + "eslint-visitor-keys": "^2.1.0", + "semver": "^6.3.0" + }, + "engines": { + "node": "^10.13.0 || ^12.13.0 || >=14.0.0" + }, + "peerDependencies": { + "@babel/core": ">=7.11.0", + "eslint": "^7.5.0 || ^8.0.0" + } + }, + "node_modules/@babel/eslint-parser/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/generator": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.17.10.tgz", + "integrity": "sha512-46MJZZo9y3o4kmhBVc7zW7i8dtR1oIK/sdO5NcfcZRhTGYi+KKJRtHNgsU6c4VUcJmUNV/LQdebD/9Dlv4K+Tg==", + "dependencies": { + "@babel/types": "^7.17.10", + "@jridgewell/gen-mapping": "^0.1.0", + "jsesc": "^2.5.1" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-annotate-as-pure": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.16.7.tgz", + "integrity": "sha512-s6t2w/IPQVTAET1HitoowRGXooX8mCgtuP5195wD/QJPV6wYjpujCGF7JuMODVX2ZAJOf1GT6DT9MHEZvLOFSw==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-builder-binary-assignment-operator-visitor": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.16.7.tgz", + "integrity": "sha512-C6FdbRaxYjwVu/geKW4ZeQ0Q31AftgRcdSnZ5/jsH6BzCJbtvXvhpfkbkThYSuutZA7nCXpPR6AD9zd1dprMkA==", + "dependencies": { + "@babel/helper-explode-assignable-expression": "^7.16.7", + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-compilation-targets": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.17.10.tgz", + "integrity": "sha512-gh3RxjWbauw/dFiU/7whjd0qN9K6nPJMqe6+Er7rOavFh0CQUSwhAE3IcTho2rywPJFxej6TUUHDkWcYI6gGqQ==", + "dependencies": { + "@babel/compat-data": "^7.17.10", + "@babel/helper-validator-option": "^7.16.7", + "browserslist": "^4.20.2", + "semver": "^6.3.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/helper-compilation-targets/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/helper-create-class-features-plugin": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.17.9.tgz", + "integrity": "sha512-kUjip3gruz6AJKOq5i3nC6CoCEEF/oHH3cp6tOZhB+IyyyPyW0g1Gfsxn3mkk6S08pIA2y8GQh609v9G/5sHVQ==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.17.9", + "@babel/helper-member-expression-to-functions": "^7.17.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/helper-create-regexp-features-plugin": { + "version": "7.17.0", + "resolved": "https://registry.npmjs.org/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.17.0.tgz", + "integrity": "sha512-awO2So99wG6KnlE+TPs6rn83gCz5WlEePJDTnLEqbchMVrBeAujURVphRdigsk094VhvZehFoNOihSlcBjwsXA==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "regexpu-core": "^5.0.1" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/helper-define-polyfill-provider": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.3.1.tgz", + "integrity": "sha512-J9hGMpJQmtWmj46B3kBHmL38UhJGhYX7eqkcq+2gsstyYt341HmPeWspihX43yVRA0mS+8GGk2Gckc7bY/HCmA==", + "dependencies": { + "@babel/helper-compilation-targets": "^7.13.0", + "@babel/helper-module-imports": "^7.12.13", + "@babel/helper-plugin-utils": "^7.13.0", + "@babel/traverse": "^7.13.0", + "debug": "^4.1.1", + "lodash.debounce": "^4.0.8", + "resolve": "^1.14.2", + "semver": "^6.1.2" + }, + "peerDependencies": { + "@babel/core": "^7.4.0-0" + } + }, + "node_modules/@babel/helper-define-polyfill-provider/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@babel/helper-define-polyfill-provider/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/helper-environment-visitor": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.16.7.tgz", + "integrity": "sha512-SLLb0AAn6PkUeAfKJCCOl9e1R53pQlGAfc4y4XuMRZfqeMYLE0dM1LMhqbGAlGQY0lfw5/ohoYWAe9V1yibRag==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-explode-assignable-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-explode-assignable-expression/-/helper-explode-assignable-expression-7.16.7.tgz", + "integrity": "sha512-KyUenhWMC8VrxzkGP0Jizjo4/Zx+1nNZhgocs+gLzyZyB8SHidhoq9KK/8Ato4anhwsivfkBLftky7gvzbZMtQ==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-function-name": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.17.9.tgz", + "integrity": "sha512-7cRisGlVtiVqZ0MW0/yFB4atgpGLWEHUVYnb448hZK4x+vih0YO5UoS11XIYtZYqHd0dIPMdUSv8q5K4LdMnIg==", + "dependencies": { + "@babel/template": "^7.16.7", + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-hoist-variables": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.16.7.tgz", + "integrity": "sha512-m04d/0Op34H5v7pbZw6pSKP7weA6lsMvfiIAMeIvkY/R4xQtBSMFEigu9QTZ2qB/9l22vsxtM8a+Q8CzD255fg==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-member-expression-to-functions": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.17.7.tgz", + "integrity": "sha512-thxXgnQ8qQ11W2wVUObIqDL4p148VMxkt5T/qpN5k2fboRyzFGFmKsTGViquyM5QHKUy48OZoca8kw4ajaDPyw==", + "dependencies": { + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-module-imports": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.16.7.tgz", + "integrity": "sha512-LVtS6TqjJHFc+nYeITRo6VLXve70xmq7wPhWTqDJusJEgGmkAACWwMiTNrvfoQo6hEhFwAIixNkvB0jPXDL8Wg==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-module-transforms": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.17.7.tgz", + "integrity": "sha512-VmZD99F3gNTYB7fJRDTi+u6l/zxY0BE6OIxPSU7a50s6ZUQkHwSDmV92FfM+oCG0pZRVojGYhkR8I0OGeCVREw==", + "dependencies": { + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-simple-access": "^7.17.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "@babel/helper-validator-identifier": "^7.16.7", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.3", + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-optimise-call-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.16.7.tgz", + "integrity": "sha512-EtgBhg7rd/JcnpZFXpBy0ze1YRfdm7BnBX4uKMBd3ixa3RGAE002JZB66FJyNH7g0F38U05pXmA5P8cBh7z+1w==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-plugin-utils": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.16.7.tgz", + "integrity": "sha512-Qg3Nk7ZxpgMrsox6HreY1ZNKdBq7K72tDSliA6dCl5f007jR4ne8iD5UzuNnCJH2xBf2BEEVGr+/OL6Gdp7RxA==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-remap-async-to-generator": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.16.8.tgz", + "integrity": "sha512-fm0gH7Flb8H51LqJHy3HJ3wnE1+qtYR2A99K06ahwrawLdOFsCEWjZOrYricXJHoPSudNKxrMBUPEIPxiIIvBw==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-wrap-function": "^7.16.8", + "@babel/types": "^7.16.8" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-replace-supers": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-replace-supers/-/helper-replace-supers-7.16.7.tgz", + "integrity": "sha512-y9vsWilTNaVnVh6xiJfABzsNpgDPKev9HnAgz6Gb1p6UUwf9NepdlsV7VXGCftJM+jqD5f7JIEubcpLjZj5dBw==", + "dependencies": { + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-member-expression-to-functions": "^7.16.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/traverse": "^7.16.7", + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-simple-access": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.17.7.tgz", + "integrity": "sha512-txyMCGroZ96i+Pxr3Je3lzEJjqwaRC9buMUgtomcrLe5Nd0+fk1h0LLA+ixUF5OW7AhHuQ7Es1WcQJZmZsz2XA==", + "dependencies": { + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-skip-transparent-expression-wrappers": { + "version": "7.16.0", + "resolved": "https://registry.npmjs.org/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.16.0.tgz", + "integrity": "sha512-+il1gTy0oHwUsBQZyJvukbB4vPMdcYBrFHa0Uc4AizLxbq6BOYC51Rv4tWocX9BLBDLZ4kc6qUFpQ6HRgL+3zw==", + "dependencies": { + "@babel/types": "^7.16.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-split-export-declaration": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.7.tgz", + "integrity": "sha512-xbWoy/PFoxSWazIToT9Sif+jJTlrMcndIsaOKvTA6u7QEo7ilkRZpjew18/W3c7nm8fXdUDXh02VXTbZ0pGDNw==", + "dependencies": { + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-validator-identifier": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.16.7.tgz", + "integrity": "sha512-hsEnFemeiW4D08A5gUAZxLBTXpZ39P+a+DGDsHw1yxqyQ/jzFEnxf5uTEGp+3bzAbNOxU1paTgYS4ECU/IgfDw==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-validator-option": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.16.7.tgz", + "integrity": "sha512-TRtenOuRUVo9oIQGPC5G9DgK4743cdxvtOw0weQNpZXaS16SCBi5MNjZF8vba3ETURjZpTbVn7Vvcf2eAwFozQ==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-wrap-function": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/helper-wrap-function/-/helper-wrap-function-7.16.8.tgz", + "integrity": "sha512-8RpyRVIAW1RcDDGTA+GpPAwV22wXCfKOoM9bet6TLkGIFTkRQSkH1nMQ5Yet4MpoXe1ZwHPVtNasc2w0uZMqnw==", + "dependencies": { + "@babel/helper-function-name": "^7.16.7", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.16.8", + "@babel/types": "^7.16.8" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helpers": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.17.9.tgz", + "integrity": "sha512-cPCt915ShDWUEzEp3+UNRktO2n6v49l5RSnG9M5pS24hA+2FAc5si+Pn1i4VVbQQ+jh+bIZhPFQOJOzbrOYY1Q==", + "dependencies": { + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.9", + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/highlight": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.17.9.tgz", + "integrity": "sha512-J9PfEKCbFIv2X5bjTMiZu6Vf341N05QIY+d6FvVKynkG1S7G0j3I0QoRtWIrXhZ+/Nlb5Q0MzqL7TokEJ5BNHg==", + "dependencies": { + "@babel/helper-validator-identifier": "^7.16.7", + "chalk": "^2.0.0", + "js-tokens": "^4.0.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/highlight/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/@babel/parser": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.17.10.tgz", + "integrity": "sha512-n2Q6i+fnJqzOaq2VkdXxy2TCPCWQZHiCo0XqmrCvDWcZQKRyZzYi4Z0yxlBuN0w+r2ZHmre+Q087DSrw3pbJDQ==", + "bin": { + "parser": "bin/babel-parser.js" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.16.7.tgz", + "integrity": "sha512-anv/DObl7waiGEnC24O9zqL0pSuI9hljihqiDuFHC8d7/bjr/4RLGPWuc8rYOff/QPzbEPSkzG8wGG9aDuhHRg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.16.7.tgz", + "integrity": "sha512-di8vUHRdf+4aJ7ltXhaDbPoszdkh59AQtJM5soLsuHpQJdFQZOA4uGj0V2u/CZ8bJ/u8ULDL5yq6FO/bCXnKHw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0", + "@babel/plugin-proposal-optional-chaining": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.13.0" + } + }, + "node_modules/@babel/plugin-proposal-async-generator-functions": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-async-generator-functions/-/plugin-proposal-async-generator-functions-7.16.8.tgz", + "integrity": "sha512-71YHIvMuiuqWJQkebWJtdhQTfd4Q4mF76q2IX37uZPkG9+olBxsX+rH1vkhFto4UeJZ9dPY2s+mDvhDm1u2BGQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-remap-async-to-generator": "^7.16.8", + "@babel/plugin-syntax-async-generators": "^7.8.4" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-class-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-class-properties/-/plugin-proposal-class-properties-7.16.7.tgz", + "integrity": "sha512-IobU0Xme31ewjYOShSIqd/ZGM/r/cuOz2z0MDbNrhF5FW+ZVgi0f2lyeoj9KFPDOAqsYxmLWZte1WOwlvY9aww==", + "dependencies": { + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-class-static-block": { + "version": "7.17.6", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-class-static-block/-/plugin-proposal-class-static-block-7.17.6.tgz", + "integrity": "sha512-X/tididvL2zbs7jZCeeRJ8167U/+Ac135AM6jCAx6gYXDUviZV5Ku9UDvWS2NCuWlFjIRXklYhwo6HhAC7ETnA==", + "dependencies": { + "@babel/helper-create-class-features-plugin": "^7.17.6", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-class-static-block": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.12.0" + } + }, + "node_modules/@babel/plugin-proposal-dynamic-import": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-dynamic-import/-/plugin-proposal-dynamic-import-7.16.7.tgz", + "integrity": "sha512-I8SW9Ho3/8DRSdmDdH3gORdyUuYnk1m4cMxUAdu5oy4n3OfN8flDEH+d60iG7dUfi0KkYwSvoalHzzdRzpWHTg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-dynamic-import": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-export-namespace-from": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-export-namespace-from/-/plugin-proposal-export-namespace-from-7.16.7.tgz", + "integrity": "sha512-ZxdtqDXLRGBL64ocZcs7ovt71L3jhC1RGSyR996svrCi3PYqHNkb3SwPJCs8RIzD86s+WPpt2S73+EHCGO+NUA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-export-namespace-from": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-json-strings": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-json-strings/-/plugin-proposal-json-strings-7.16.7.tgz", + "integrity": "sha512-lNZ3EEggsGY78JavgbHsK9u5P3pQaW7k4axlgFLYkMd7UBsiNahCITShLjNQschPyjtO6dADrL24757IdhBrsQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-json-strings": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-logical-assignment-operators": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-logical-assignment-operators/-/plugin-proposal-logical-assignment-operators-7.16.7.tgz", + "integrity": "sha512-K3XzyZJGQCr00+EtYtrDjmwX7o7PLK6U9bi1nCwkQioRFVUv6dJoxbQjtWVtP+bCPy82bONBKG8NPyQ4+i6yjg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-logical-assignment-operators": "^7.10.4" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-nullish-coalescing-operator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-nullish-coalescing-operator/-/plugin-proposal-nullish-coalescing-operator-7.16.7.tgz", + "integrity": "sha512-aUOrYU3EVtjf62jQrCj63pYZ7k6vns2h/DQvHPWGmsJRYzWXZ6/AsfgpiRy6XiuIDADhJzP2Q9MwSMKauBQ+UQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-numeric-separator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-numeric-separator/-/plugin-proposal-numeric-separator-7.16.7.tgz", + "integrity": "sha512-vQgPMknOIgiuVqbokToyXbkY/OmmjAzr/0lhSIbG/KmnzXPGwW/AdhdKpi+O4X/VkWiWjnkKOBiqJrTaC98VKw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-numeric-separator": "^7.10.4" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-object-rest-spread": { + "version": "7.17.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-object-rest-spread/-/plugin-proposal-object-rest-spread-7.17.3.tgz", + "integrity": "sha512-yuL5iQA/TbZn+RGAfxQXfi7CNLmKi1f8zInn4IgobuCWcAb7i+zj4TYzQ9l8cEzVyJ89PDGuqxK1xZpUDISesw==", + "dependencies": { + "@babel/compat-data": "^7.17.0", + "@babel/helper-compilation-targets": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-object-rest-spread": "^7.8.3", + "@babel/plugin-transform-parameters": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-optional-catch-binding": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-optional-catch-binding/-/plugin-proposal-optional-catch-binding-7.16.7.tgz", + "integrity": "sha512-eMOH/L4OvWSZAE1VkHbr1vckLG1WUcHGJSLqqQwl2GaUqG6QjddvrOaTUMNYiv77H5IKPMZ9U9P7EaHwvAShfA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-optional-catch-binding": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-optional-chaining": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-optional-chaining/-/plugin-proposal-optional-chaining-7.16.7.tgz", + "integrity": "sha512-eC3xy+ZrUcBtP7x+sq62Q/HYd674pPTb/77XZMb5wbDPGWIdUbSr4Agr052+zaUPSb+gGRnjxXfKFvx5iMJ+DA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0", + "@babel/plugin-syntax-optional-chaining": "^7.8.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-private-methods": { + "version": "7.16.11", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-private-methods/-/plugin-proposal-private-methods-7.16.11.tgz", + "integrity": "sha512-F/2uAkPlXDr8+BHpZvo19w3hLFKge+k75XUprE6jaqKxjGkSYcK+4c+bup5PdW/7W/Rpjwql7FTVEDW+fRAQsw==", + "dependencies": { + "@babel/helper-create-class-features-plugin": "^7.16.10", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-private-property-in-object": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.16.7.tgz", + "integrity": "sha512-rMQkjcOFbm+ufe3bTZLyOfsOUOxyvLXZJCTARhJr+8UMSoZmqTe1K1BgkFcrW37rAchWg57yI69ORxiWvUINuQ==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-private-property-in-object": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-proposal-unicode-property-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-unicode-property-regex/-/plugin-proposal-unicode-property-regex-7.16.7.tgz", + "integrity": "sha512-QRK0YI/40VLhNVGIjRNAAQkEHws0cswSdFFjpFyt943YmJIU1da9uW63Iu6NFV6CxTZW5eTDCrwZUstBWgp/Rg==", + "dependencies": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-async-generators": { + "version": "7.8.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz", + "integrity": "sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-class-properties": { + "version": "7.12.13", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz", + "integrity": "sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.12.13" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-class-static-block": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz", + "integrity": "sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-dynamic-import": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz", + "integrity": "sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-export-namespace-from": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz", + "integrity": "sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.3" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-flow": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-flow/-/plugin-syntax-flow-7.16.7.tgz", + "integrity": "sha512-UDo3YGQO0jH6ytzVwgSLv9i/CzMcUjbKenL67dTrAZPPv6GFAtDhe6jqnvmoKzC/7htNTohhos+onPtDMqJwaQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-json-strings": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz", + "integrity": "sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-jsx": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.16.7.tgz", + "integrity": "sha512-Esxmk7YjA8QysKeT3VhTXvF6y77f/a91SIs4pWb4H2eWGQkCKFgQaG6hdoEVZtGsrAcb2K5BW66XsOErD4WU3Q==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-logical-assignment-operators": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz", + "integrity": "sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-nullish-coalescing-operator": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz", + "integrity": "sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-numeric-separator": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz", + "integrity": "sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-object-rest-spread": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz", + "integrity": "sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-optional-catch-binding": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz", + "integrity": "sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-optional-chaining": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz", + "integrity": "sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-private-property-in-object": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz", + "integrity": "sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-top-level-await": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz", + "integrity": "sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-typescript": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.17.10.tgz", + "integrity": "sha512-xJefea1DWXW09pW4Tm9bjwVlPDyYA2it3fWlmEjpYz6alPvTUjL0EOzNzI/FEOyI3r4/J7uVH5UqKgl1TQ5hqQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-arrow-functions": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.16.7.tgz", + "integrity": "sha512-9ffkFFMbvzTvv+7dTp/66xvZAWASuPD5Tl9LK3Z9vhOmANo6j94rik+5YMBt4CwHVMWLWpMsriIc2zsa3WW3xQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-async-to-generator": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.16.8.tgz", + "integrity": "sha512-MtmUmTJQHCnyJVrScNzNlofQJ3dLFuobYn3mwOTKHnSCMtbNsqvF71GQmJfFjdrXSsAA7iysFmYWw4bXZ20hOg==", + "dependencies": { + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-remap-async-to-generator": "^7.16.8" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-block-scoped-functions": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.16.7.tgz", + "integrity": "sha512-JUuzlzmF40Z9cXyytcbZEZKckgrQzChbQJw/5PuEHYeqzCsvebDx0K0jWnIIVcmmDOAVctCgnYs0pMcrYj2zJg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-block-scoping": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.16.7.tgz", + "integrity": "sha512-ObZev2nxVAYA4bhyusELdo9hb3H+A56bxH3FZMbEImZFiEDYVHXQSJ1hQKFlDnlt8G9bBrCZ5ZpURZUrV4G5qQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-classes": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-classes/-/plugin-transform-classes-7.16.7.tgz", + "integrity": "sha512-WY7og38SFAGYRe64BrjKf8OrE6ulEHtr5jEYaZMwox9KebgqPi67Zqz8K53EKk1fFEJgm96r32rkKZ3qA2nCWQ==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.16.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "globals": "^11.1.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-computed-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.16.7.tgz", + "integrity": "sha512-gN72G9bcmenVILj//sv1zLNaPyYcOzUho2lIJBMh/iakJ9ygCo/hEF9cpGb61SCMEDxbbyBoVQxrt+bWKu5KGw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-destructuring": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.17.7.tgz", + "integrity": "sha512-XVh0r5yq9sLR4vZ6eVZe8FKfIcSgaTBxVBRSYokRj2qksf6QerYnTxz9/GTuKTH/n/HwLP7t6gtlybHetJ/6hQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-dotall-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.16.7.tgz", + "integrity": "sha512-Lyttaao2SjZF6Pf4vk1dVKv8YypMpomAbygW+mU5cYP3S5cWTfCJjG8xV6CFdzGFlfWK81IjL9viiTvpb6G7gQ==", + "dependencies": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-duplicate-keys": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.16.7.tgz", + "integrity": "sha512-03DvpbRfvWIXyK0/6QiR1KMTWeT6OcQ7tbhjrXyFS02kjuX/mu5Bvnh5SDSWHxyawit2g5aWhKwI86EE7GUnTw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-exponentiation-operator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.16.7.tgz", + "integrity": "sha512-8UYLSlyLgRixQvlYH3J2ekXFHDFLQutdy7FfFAMm3CPZ6q9wHCwnUyiXpQCe3gVVnQlHc5nsuiEVziteRNTXEA==", + "dependencies": { + "@babel/helper-builder-binary-assignment-operator-visitor": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-flow-strip-types": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-flow-strip-types/-/plugin-transform-flow-strip-types-7.16.7.tgz", + "integrity": "sha512-mzmCq3cNsDpZZu9FADYYyfZJIOrSONmHcop2XEKPdBNMa4PDC4eEvcOvzZaCNcjKu72v0XQlA5y1g58aLRXdYg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-flow": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-for-of": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.16.7.tgz", + "integrity": "sha512-/QZm9W92Ptpw7sjI9Nx1mbcsWz33+l8kuMIQnDwgQBG5s3fAfQvkRjQ7NqXhtNcKOnPkdICmUHyCaWW06HCsqg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-function-name": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.16.7.tgz", + "integrity": "sha512-SU/C68YVwTRxqWj5kgsbKINakGag0KTgq9f2iZEXdStoAbOzLHEBRYzImmA6yFo8YZhJVflvXmIHUO7GWHmxxA==", + "dependencies": { + "@babel/helper-compilation-targets": "^7.16.7", + "@babel/helper-function-name": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-literals/-/plugin-transform-literals-7.16.7.tgz", + "integrity": "sha512-6tH8RTpTWI0s2sV6uq3e/C9wPo4PTqqZps4uF0kzQ9/xPLFQtipynvmT1g/dOfEJ+0EQsHhkQ/zyRId8J2b8zQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-member-expression-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.16.7.tgz", + "integrity": "sha512-mBruRMbktKQwbxaJof32LT9KLy2f3gH+27a5XSuXo6h7R3vqltl0PgZ80C8ZMKw98Bf8bqt6BEVi3svOh2PzMw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-modules-amd": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.16.7.tgz", + "integrity": "sha512-KaaEtgBL7FKYwjJ/teH63oAmE3lP34N3kshz8mm4VMAw7U3PxjVwwUmxEFksbgsNUaO3wId9R2AVQYSEGRa2+g==", + "dependencies": { + "@babel/helper-module-transforms": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-modules-commonjs": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.17.9.tgz", + "integrity": "sha512-2TBFd/r2I6VlYn0YRTz2JdazS+FoUuQ2rIFHoAxtyP/0G3D82SBLaRq9rnUkpqlLg03Byfl/+M32mpxjO6KaPw==", + "dependencies": { + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-simple-access": "^7.17.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-modules-systemjs": { + "version": "7.17.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.17.8.tgz", + "integrity": "sha512-39reIkMTUVagzgA5x88zDYXPCMT6lcaRKs1+S9K6NKBPErbgO/w/kP8GlNQTC87b412ZTlmNgr3k2JrWgHH+Bw==", + "dependencies": { + "@babel/helper-hoist-variables": "^7.16.7", + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-identifier": "^7.16.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-modules-umd": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.16.7.tgz", + "integrity": "sha512-EMh7uolsC8O4xhudF2F6wedbSHm1HHZ0C6aJ7K67zcDNidMzVcxWdGr+htW9n21klm+bOn+Rx4CBsAntZd3rEQ==", + "dependencies": { + "@babel/helper-module-transforms": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-named-capturing-groups-regex": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.17.10.tgz", + "integrity": "sha512-v54O6yLaJySCs6mGzaVOUw9T967GnH38T6CQSAtnzdNPwu84l2qAjssKzo/WSO8Yi7NF+7ekm5cVbF/5qiIgNA==", + "dependencies": { + "@babel/helper-create-regexp-features-plugin": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/plugin-transform-new-target": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.16.7.tgz", + "integrity": "sha512-xiLDzWNMfKoGOpc6t3U+etCE2yRnn3SM09BXqWPIZOBpL2gvVrBWUKnsJx0K/ADi5F5YC5f8APFfWrz25TdlGg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-object-super": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.16.7.tgz", + "integrity": "sha512-14J1feiQVWaGvRxj2WjyMuXS2jsBkgB3MdSN5HuC2G5nRspa5RK9COcs82Pwy5BuGcjb+fYaUj94mYcOj7rCvw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-parameters": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.16.7.tgz", + "integrity": "sha512-AT3MufQ7zZEhU2hwOA11axBnExW0Lszu4RL/tAlUJBuNoRak+wehQW8h6KcXOcgjY42fHtDxswuMhMjFEuv/aw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-property-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.16.7.tgz", + "integrity": "sha512-z4FGr9NMGdoIl1RqavCqGG+ZuYjfZ/hkCIeuH6Do7tXmSm0ls11nYVSJqFEUOSJbDab5wC6lRE/w6YjVcr6Hqw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-react-display-name": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.16.7.tgz", + "integrity": "sha512-qgIg8BcZgd0G/Cz916D5+9kqX0c7nPZyXaP8R2tLNN5tkyIZdG5fEwBrxwplzSnjC1jvQmyMNVwUCZPcbGY7Pg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-react-jsx": { + "version": "7.17.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.17.3.tgz", + "integrity": "sha512-9tjBm4O07f7mzKSIlEmPdiE6ub7kfIe6Cd+w+oQebpATfTQMAgW+YOuWxogbKVTulA+MEO7byMeIUtQ1z+z+ZQ==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-jsx": "^7.16.7", + "@babel/types": "^7.17.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-react-jsx-development": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.16.7.tgz", + "integrity": "sha512-RMvQWvpla+xy6MlBpPlrKZCMRs2AGiHOGHY3xRwl0pEeim348dDyxeH4xBsMPbIMhujeq7ihE702eM2Ew0Wo+A==", + "dependencies": { + "@babel/plugin-transform-react-jsx": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-react-pure-annotations": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.16.7.tgz", + "integrity": "sha512-hs71ToC97k3QWxswh2ElzMFABXHvGiJ01IB1TbYQDGeWRKWz/MPUTh5jGExdHvosYKpnJW5Pm3S4+TA3FyX+GA==", + "dependencies": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-regenerator": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.17.9.tgz", + "integrity": "sha512-Lc2TfbxR1HOyn/c6b4Y/b6NHoTb67n/IoWLxTu4kC7h4KQnWlhCq2S8Tx0t2SVvv5Uu87Hs+6JEJ5kt2tYGylQ==", + "dependencies": { + "regenerator-transform": "^0.15.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-reserved-words": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.16.7.tgz", + "integrity": "sha512-KQzzDnZ9hWQBjwi5lpY5v9shmm6IVG0U9pB18zvMu2i4H90xpT4gmqwPYsn8rObiadYe2M0gmgsiOIF5A/2rtg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-runtime": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.17.10.tgz", + "integrity": "sha512-6jrMilUAJhktTr56kACL8LnWC5hx3Lf27BS0R0DSyW/OoJfb/iTHeE96V3b1dgKG3FSFdd/0culnYWMkjcKCig==", + "dependencies": { + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "babel-plugin-polyfill-corejs2": "^0.3.0", + "babel-plugin-polyfill-corejs3": "^0.5.0", + "babel-plugin-polyfill-regenerator": "^0.3.0", + "semver": "^6.3.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-runtime/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/plugin-transform-shorthand-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.16.7.tgz", + "integrity": "sha512-hah2+FEnoRoATdIb05IOXf+4GzXYTq75TVhIn1PewihbpyrNWUt2JbudKQOETWw6QpLe+AIUpJ5MVLYTQbeeUg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-spread": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-spread/-/plugin-transform-spread-7.16.7.tgz", + "integrity": "sha512-+pjJpgAngb53L0iaA5gU/1MLXJIfXcYepLgXB3esVRf4fqmj8f2cxM3/FKaHsZms08hFQJkFccEWuIpm429TXg==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-sticky-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.16.7.tgz", + "integrity": "sha512-NJa0Bd/87QV5NZZzTuZG5BPJjLYadeSZ9fO6oOUoL4iQx+9EEuw/eEM92SrsT19Yc2jgB1u1hsjqDtH02c3Drw==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-template-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.16.7.tgz", + "integrity": "sha512-VwbkDDUeenlIjmfNeDX/V0aWrQH2QiVyJtwymVQSzItFDTpxfyJh3EVaQiS0rIN/CqbLGr0VcGmuwyTdZtdIsA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-typeof-symbol": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.16.7.tgz", + "integrity": "sha512-p2rOixCKRJzpg9JB4gjnG4gjWkWa89ZoYUnl9snJ1cWIcTH/hvxZqfO+WjG6T8DRBpctEol5jw1O5rA8gkCokQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-typescript": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.16.8.tgz", + "integrity": "sha512-bHdQ9k7YpBDO2d0NVfkj51DpQcvwIzIusJ7mEUaMlbZq3Kt/U47j24inXZHQ5MDiYpCs+oZiwnXyKedE8+q7AQ==", + "dependencies": { + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-typescript": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-unicode-escapes": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.16.7.tgz", + "integrity": "sha512-TAV5IGahIz3yZ9/Hfv35TV2xEm+kaBDaZQCn2S/hG9/CZ0DktxJv9eKfPc7yYCvOYR4JGx1h8C+jcSOvgaaI/Q==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-transform-unicode-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.16.7.tgz", + "integrity": "sha512-oC5tYYKw56HO75KZVLQ+R/Nl3Hro9kf8iG0hXoaHP7tjAyCpvqBiSNe6vGrZni1Z6MggmUOC6A7VP7AVmw225Q==", + "dependencies": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/preset-env": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/preset-env/-/preset-env-7.17.10.tgz", + "integrity": "sha512-YNgyBHZQpeoBSRBg0xixsZzfT58Ze1iZrajvv0lJc70qDDGuGfonEnMGfWeSY0mQ3JTuCWFbMkzFRVafOyJx4g==", + "dependencies": { + "@babel/compat-data": "^7.17.10", + "@babel/helper-compilation-targets": "^7.17.10", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression": "^7.16.7", + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining": "^7.16.7", + "@babel/plugin-proposal-async-generator-functions": "^7.16.8", + "@babel/plugin-proposal-class-properties": "^7.16.7", + "@babel/plugin-proposal-class-static-block": "^7.17.6", + "@babel/plugin-proposal-dynamic-import": "^7.16.7", + "@babel/plugin-proposal-export-namespace-from": "^7.16.7", + "@babel/plugin-proposal-json-strings": "^7.16.7", + "@babel/plugin-proposal-logical-assignment-operators": "^7.16.7", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.16.7", + "@babel/plugin-proposal-numeric-separator": "^7.16.7", + "@babel/plugin-proposal-object-rest-spread": "^7.17.3", + "@babel/plugin-proposal-optional-catch-binding": "^7.16.7", + "@babel/plugin-proposal-optional-chaining": "^7.16.7", + "@babel/plugin-proposal-private-methods": "^7.16.11", + "@babel/plugin-proposal-private-property-in-object": "^7.16.7", + "@babel/plugin-proposal-unicode-property-regex": "^7.16.7", + "@babel/plugin-syntax-async-generators": "^7.8.4", + "@babel/plugin-syntax-class-properties": "^7.12.13", + "@babel/plugin-syntax-class-static-block": "^7.14.5", + "@babel/plugin-syntax-dynamic-import": "^7.8.3", + "@babel/plugin-syntax-export-namespace-from": "^7.8.3", + "@babel/plugin-syntax-json-strings": "^7.8.3", + "@babel/plugin-syntax-logical-assignment-operators": "^7.10.4", + "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3", + "@babel/plugin-syntax-numeric-separator": "^7.10.4", + "@babel/plugin-syntax-object-rest-spread": "^7.8.3", + "@babel/plugin-syntax-optional-catch-binding": "^7.8.3", + "@babel/plugin-syntax-optional-chaining": "^7.8.3", + "@babel/plugin-syntax-private-property-in-object": "^7.14.5", + "@babel/plugin-syntax-top-level-await": "^7.14.5", + "@babel/plugin-transform-arrow-functions": "^7.16.7", + "@babel/plugin-transform-async-to-generator": "^7.16.8", + "@babel/plugin-transform-block-scoped-functions": "^7.16.7", + "@babel/plugin-transform-block-scoping": "^7.16.7", + "@babel/plugin-transform-classes": "^7.16.7", + "@babel/plugin-transform-computed-properties": "^7.16.7", + "@babel/plugin-transform-destructuring": "^7.17.7", + "@babel/plugin-transform-dotall-regex": "^7.16.7", + "@babel/plugin-transform-duplicate-keys": "^7.16.7", + "@babel/plugin-transform-exponentiation-operator": "^7.16.7", + "@babel/plugin-transform-for-of": "^7.16.7", + "@babel/plugin-transform-function-name": "^7.16.7", + "@babel/plugin-transform-literals": "^7.16.7", + "@babel/plugin-transform-member-expression-literals": "^7.16.7", + "@babel/plugin-transform-modules-amd": "^7.16.7", + "@babel/plugin-transform-modules-commonjs": "^7.17.9", + "@babel/plugin-transform-modules-systemjs": "^7.17.8", + "@babel/plugin-transform-modules-umd": "^7.16.7", + "@babel/plugin-transform-named-capturing-groups-regex": "^7.17.10", + "@babel/plugin-transform-new-target": "^7.16.7", + "@babel/plugin-transform-object-super": "^7.16.7", + "@babel/plugin-transform-parameters": "^7.16.7", + "@babel/plugin-transform-property-literals": "^7.16.7", + "@babel/plugin-transform-regenerator": "^7.17.9", + "@babel/plugin-transform-reserved-words": "^7.16.7", + "@babel/plugin-transform-shorthand-properties": "^7.16.7", + "@babel/plugin-transform-spread": "^7.16.7", + "@babel/plugin-transform-sticky-regex": "^7.16.7", + "@babel/plugin-transform-template-literals": "^7.16.7", + "@babel/plugin-transform-typeof-symbol": "^7.16.7", + "@babel/plugin-transform-unicode-escapes": "^7.16.7", + "@babel/plugin-transform-unicode-regex": "^7.16.7", + "@babel/preset-modules": "^0.1.5", + "@babel/types": "^7.17.10", + "babel-plugin-polyfill-corejs2": "^0.3.0", + "babel-plugin-polyfill-corejs3": "^0.5.0", + "babel-plugin-polyfill-regenerator": "^0.3.0", + "core-js-compat": "^3.22.1", + "semver": "^6.3.0" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/preset-env/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/preset-modules": { + "version": "0.1.5", + "resolved": "https://registry.npmjs.org/@babel/preset-modules/-/preset-modules-0.1.5.tgz", + "integrity": "sha512-A57th6YRG7oR3cq/yt/Y84MvGgE0eJG2F1JLhKuyG+jFxEgrd/HAMJatiFtmOiZurz+0DkrvbheCLaV5f2JfjA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.0.0", + "@babel/plugin-proposal-unicode-property-regex": "^7.4.4", + "@babel/plugin-transform-dotall-regex": "^7.4.4", + "@babel/types": "^7.4.4", + "esutils": "^2.0.2" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/preset-react": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/preset-react/-/preset-react-7.16.7.tgz", + "integrity": "sha512-fWpyI8UM/HE6DfPBzD8LnhQ/OcH8AgTaqcqP2nGOXEUV+VKBR5JRN9hCk9ai+zQQ57vtm9oWeXguBCPNUjytgA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-transform-react-display-name": "^7.16.7", + "@babel/plugin-transform-react-jsx": "^7.16.7", + "@babel/plugin-transform-react-jsx-development": "^7.16.7", + "@babel/plugin-transform-react-pure-annotations": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/preset-typescript": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/preset-typescript/-/preset-typescript-7.16.7.tgz", + "integrity": "sha512-WbVEmgXdIyvzB77AQjGBEyYPZx+8tTsO50XtfozQrkW8QB2rLJpH2lgx0TRw5EJrBxOZQ+wCcyPVQvS8tjEHpQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-transform-typescript": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/runtime": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.17.9.tgz", + "integrity": "sha512-lSiBBvodq29uShpWGNbgFdKYNiFDo5/HIYsaCEY9ff4sb10x9jizo2+pRrSyF4jKZCXqgzuqBOQKbUm90gQwJg==", + "dependencies": { + "regenerator-runtime": "^0.13.4" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/runtime-corejs3": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/runtime-corejs3/-/runtime-corejs3-7.17.9.tgz", + "integrity": "sha512-WxYHHUWF2uZ7Hp1K+D1xQgbgkGUfA+5UPOegEXGt2Y5SMog/rYCVaifLZDbw8UkNXozEqqrZTy6bglL7xTaCOw==", + "dependencies": { + "core-js-pure": "^3.20.2", + "regenerator-runtime": "^0.13.4" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/template": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.16.7.tgz", + "integrity": "sha512-I8j/x8kHUrbYRTUxXrrMbfCa7jxkE7tZre39x3kjr9hvI82cK1FfqLygotcWN5kdPGWcLdWMHpSBavse5tWw3w==", + "dependencies": { + "@babel/code-frame": "^7.16.7", + "@babel/parser": "^7.16.7", + "@babel/types": "^7.16.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/traverse": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.17.10.tgz", + "integrity": "sha512-VmbrTHQteIdUUQNTb+zE12SHS/xQVIShmBPhlNP12hD5poF2pbITW1Z4172d03HegaQWhLffdkRJYtAzp0AGcw==", + "dependencies": { + "@babel/code-frame": "^7.16.7", + "@babel/generator": "^7.17.10", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.17.9", + "@babel/helper-hoist-variables": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "@babel/parser": "^7.17.10", + "@babel/types": "^7.17.10", + "debug": "^4.1.0", + "globals": "^11.1.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/traverse/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@babel/types": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.17.10.tgz", + "integrity": "sha512-9O26jG0mBYfGkUYCYZRnBwbVLd1UZOICEr2Em6InB6jVfsAv1GKgwXHmrSg+WFWDmeKTA6vyTZiN8tCSM5Oo3A==", + "dependencies": { + "@babel/helper-validator-identifier": "^7.16.7", + "to-fast-properties": "^2.0.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@emotion/cache": { + "version": "11.7.1", + "resolved": "https://registry.npmmirror.com/@emotion/cache/-/cache-11.7.1.tgz", + "integrity": "sha512-r65Zy4Iljb8oyjtLeCuBH8Qjiy107dOYC6SJq7g7GV5UCQWMObY4SJDPGFjiiVpPrOJ2hmJOoBiYTC7hwx9E2A==", + "dependencies": { + "@emotion/memoize": "^0.7.4", + "@emotion/sheet": "^1.1.0", + "@emotion/utils": "^1.0.0", + "@emotion/weak-memoize": "^0.2.5", + "stylis": "4.0.13" + } + }, + "node_modules/@emotion/hash": { + "version": "0.8.0", + "resolved": "https://registry.npmmirror.com/@emotion/hash/-/hash-0.8.0.tgz", + "integrity": "sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow==" + }, + "node_modules/@emotion/memoize": { + "version": "0.7.5", + "resolved": "https://registry.npmmirror.com/@emotion/memoize/-/memoize-0.7.5.tgz", + "integrity": "sha512-igX9a37DR2ZPGYtV6suZ6whr8pTFtyHL3K/oLUotxpSVO2ASaprmAe2Dkq7tBo7CRY7MMDrAa9nuQP9/YG8FxQ==" + }, + "node_modules/@emotion/react": { + "version": "11.7.1", + "resolved": "https://registry.npmmirror.com/@emotion/react/-/react-11.7.1.tgz", + "integrity": "sha512-DV2Xe3yhkF1yT4uAUoJcYL1AmrnO5SVsdfvu+fBuS7IbByDeTVx9+wFmvx9Idzv7/78+9Mgx2Hcmr7Fex3tIyw==", + "dependencies": { + "@babel/runtime": "^7.13.10", + "@emotion/cache": "^11.7.1", + "@emotion/serialize": "^1.0.2", + "@emotion/sheet": "^1.1.0", + "@emotion/utils": "^1.0.0", + "@emotion/weak-memoize": "^0.2.5", + "hoist-non-react-statics": "^3.3.1" + }, + "peerDependencies": { + "@babel/core": "^7.0.0", + "react": ">=16.8.0" + }, + "peerDependenciesMeta": { + "@babel/core": { + "optional": true + }, + "@types/react": { + "optional": true + } + } + }, + "node_modules/@emotion/serialize": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/@emotion/serialize/-/serialize-1.0.2.tgz", + "integrity": "sha512-95MgNJ9+/ajxU7QIAruiOAdYNjxZX7G2mhgrtDWswA21VviYIRP1R5QilZ/bDY42xiKsaktP4egJb3QdYQZi1A==", + "dependencies": { + "@emotion/hash": "^0.8.0", + "@emotion/memoize": "^0.7.4", + "@emotion/unitless": "^0.7.5", + "@emotion/utils": "^1.0.0", + "csstype": "^3.0.2" + } + }, + "node_modules/@emotion/server": { + "version": "11.4.0", + "resolved": "https://registry.npmmirror.com/@emotion/server/-/server-11.4.0.tgz", + "integrity": "sha512-IHovdWA3V0DokzxLtUNDx4+hQI82zUXqQFcVz/om2t44O0YSc+NHB+qifnyAOoQwt3SXcBTgaSntobwUI9gnfA==", + "dependencies": { + "@emotion/utils": "^1.0.0", + "html-tokenize": "^2.0.0", + "multipipe": "^1.0.2", + "through": "^2.3.8" + }, + "peerDependencies": { + "@emotion/css": "^11.0.0-rc.0" + }, + "peerDependenciesMeta": { + "@emotion/css": { + "optional": true + } + } + }, + "node_modules/@emotion/sheet": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/@emotion/sheet/-/sheet-1.1.0.tgz", + "integrity": "sha512-u0AX4aSo25sMAygCuQTzS+HsImZFuS8llY8O7b9MDRzbJM0kVJlAz6KNDqcG7pOuQZJmj/8X/rAW+66kMnMW+g==" + }, + "node_modules/@emotion/unitless": { + "version": "0.7.5", + "resolved": "https://registry.npmmirror.com/@emotion/unitless/-/unitless-0.7.5.tgz", + "integrity": "sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg==" + }, + "node_modules/@emotion/utils": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/@emotion/utils/-/utils-1.0.0.tgz", + "integrity": "sha512-mQC2b3XLDs6QCW+pDQDiyO/EdGZYOygE8s5N5rrzjSI4M3IejPE/JPndCBwRT9z982aqQNi6beWs1UeayrQxxA==" + }, + "node_modules/@emotion/weak-memoize": { + "version": "0.2.5", + "resolved": "https://registry.npmmirror.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz", + "integrity": "sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA==" + }, + "node_modules/@endemolshinegroup/cosmiconfig-typescript-loader": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/@endemolshinegroup/cosmiconfig-typescript-loader/-/cosmiconfig-typescript-loader-3.0.2.tgz", + "integrity": "sha512-QRVtqJuS1mcT56oHpVegkKBlgtWjXw/gHNWO3eL9oyB5Sc7HBoc2OLG/nYpVfT/Jejvo3NUrD0Udk7XgoyDKkA==", + "dependencies": { + "lodash.get": "^4", + "make-error": "^1", + "ts-node": "^9", + "tslib": "^2" + }, + "engines": { + "node": ">=10.0.0" + }, + "peerDependencies": { + "cosmiconfig": ">=6" + } + }, + "node_modules/@eslint/eslintrc": { + "version": "0.4.3", + "resolved": "https://registry.npmjs.org/@eslint/eslintrc/-/eslintrc-0.4.3.tgz", + "integrity": "sha512-J6KFFz5QCYUJq3pf0mjEcCJVERbzv71PUIDczuh9JkwGEzced6CO5ADLHB1rbf/+oPBtoPfMYNOpGDzCANlbXw==", + "dependencies": { + "ajv": "^6.12.4", + "debug": "^4.1.1", + "espree": "^7.3.0", + "globals": "^13.9.0", + "ignore": "^4.0.6", + "import-fresh": "^3.2.1", + "js-yaml": "^3.13.1", + "minimatch": "^3.0.4", + "strip-json-comments": "^3.1.1" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/@eslint/eslintrc/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@eslint/eslintrc/node_modules/globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "dependencies": { + "type-fest": "^0.20.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@eslint/eslintrc/node_modules/ignore": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-4.0.6.tgz", + "integrity": "sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg==", + "engines": { + "node": ">= 4" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd": { + "version": "0.0.2", + "resolved": "https://registry.npmjs.org/@gatsbyjs/parcel-namer-relative-to-cwd/-/parcel-namer-relative-to-cwd-0.0.2.tgz", + "integrity": "sha512-ZeGxCbx13+zjpE/0HuJ/tjox9zfiYq9fGoAAi+RHP5vHSJCmJVO5hZbexQ/umlUyAkkkzC4p1WIpw1cYQTA8SA==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@parcel/plugin": "2.3.1", + "gatsby-core-utils": "^3.8.2" + }, + "engines": { + "node": ">=14.15.0", + "parcel": "2.x" + }, + "peerDependencies": { + "@parcel/namer-default": "^2.3.1" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/cache": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/cache/-/cache-2.3.1.tgz", + "integrity": "sha512-8Wvm0VERtocUepIfkZ6xVs1LHZqttnzdrM7oSc0bXhwtz8kZB++N88g0rQskbUchW87314eYdzBtEL0aiq0bgQ==", + "dependencies": { + "@parcel/fs": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/utils": "2.3.1", + "lmdb": "^2.0.2" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.3.1" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/codeframe": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/codeframe/-/codeframe-2.3.1.tgz", + "integrity": "sha512-sdNvbg9qYS2pwzqyyyt+wZfNGuy7EslzDLbzQclFZmhD6e770mcYoi8/7i7D/AONbXiI15vwNmgOdcUIXtPxbA==", + "dependencies": { + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/diagnostic": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/diagnostic/-/diagnostic-2.3.1.tgz", + "integrity": "sha512-hBMcg4WVMdSIy6RpI4gSto5dZ3OoUbnrCZzVw3J1tzQJn7x9na/+014IaE58vJtAqJ8/jc/TqWIcwsSLe898rA==", + "dependencies": { + "json-source-map": "^0.6.1", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/events": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/events/-/events-2.3.1.tgz", + "integrity": "sha512-J2rWKGl1Z2IvwwDwWYz/4gUxC1P4LsioUyOo1HYGT+N5+r41P8ZB5CM/aosI2qu5mMsH8rTpclOv5E36vCSQxw==", + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/fs": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/fs/-/fs-2.3.1.tgz", + "integrity": "sha512-FKqyf8KF0zOw8gfj/feEAMj4Kzqkgt9Zxa2A7UDdMWRvxLR8znqnWjD++xqq6rxJp2Y1zm4fH3JOTK4CRddUSg==", + "dependencies": { + "@parcel/fs-search": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "@parcel/watcher": "^2.0.0", + "@parcel/workers": "2.3.1" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.3.1" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/fs-search": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/fs-search/-/fs-search-2.3.1.tgz", + "integrity": "sha512-JsBIDttjmgJIMD6Q6MV83M+mwr5NqUm55iA+SewimboiWzSPzIJxRaegniSsNfsrBASJ6nSZFHcLPd/VJ5iqJw==", + "dependencies": { + "detect-libc": "^1.0.3" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/hash": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/hash/-/hash-2.3.1.tgz", + "integrity": "sha512-IYhSQE+CIKWjPfiLmsrXHupkNd+hMlTlI9DR5qLiD8ydyPwg0XE/bOYTcbdsSl6HTackY0XYVSJwTtEgvtYVfw==", + "dependencies": { + "detect-libc": "^1.0.3", + "xxhash-wasm": "^0.4.2" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/logger": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/logger/-/logger-2.3.1.tgz", + "integrity": "sha512-swNPInULCJrpCJCLOgZcf+xNcUF0NjD7LyNcB349BkyO7i6st14nfBjXf6eAJJu0z7RMmi6zp9CQB47e4cI6+g==", + "dependencies": { + "@parcel/diagnostic": "2.3.1", + "@parcel/events": "2.3.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/markdown-ansi": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/markdown-ansi/-/markdown-ansi-2.3.1.tgz", + "integrity": "sha512-M4Hi25pKtSh1KF/ppMDBk5QuLpYAQjgB/MSP+nz7NzXQlYPCN5oEk9TUkrmQ9J+vOvVwefxfy7ahSErEuQbTFw==", + "dependencies": { + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/package-manager": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/package-manager/-/package-manager-2.3.1.tgz", + "integrity": "sha512-w2XOkD3SU8RxhUDW+Soy/TjvEVvfUsBmHy02asllt4b/ZtyZVAsQmonGExHDDkRn3TNDR6Y96Yw6M7purt+b9w==", + "dependencies": { + "@parcel/diagnostic": "2.3.1", + "@parcel/fs": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "@parcel/workers": "2.3.1", + "semver": "^5.7.1" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.3.1" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/plugin": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/plugin/-/plugin-2.3.1.tgz", + "integrity": "sha512-ROOWbgFze7BCF3RkEh8VbcKGlR5UGBuJ8lfCaFrG1VOk7Rxgl8Bmk96TRbZREm/1jB74p2O8twVKyPSC13riow==", + "dependencies": { + "@parcel/types": "2.3.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/types": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/types/-/types-2.3.1.tgz", + "integrity": "sha512-i2UyUoA4DzyYxe9rZRDuMAZ6TD3Mq3tTTqeJ2/zA6w83Aon3cqdE9va91peu1fKRGyRqE5lwWRtA7ktF1A2SVA==", + "dependencies": { + "@parcel/cache": "2.3.1", + "@parcel/diagnostic": "2.3.1", + "@parcel/fs": "2.3.1", + "@parcel/package-manager": "2.3.1", + "@parcel/source-map": "^2.0.0", + "@parcel/workers": "2.3.1", + "utility-types": "^3.10.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/utils": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/utils/-/utils-2.3.1.tgz", + "integrity": "sha512-OFdh/HuAcce753/U3QoORzYU3N5oZqCfQNRb0i3onuz/qpli5TyxUl/k1BuTqlKYr6Px3kj05g6GFi9kRBOMbw==", + "dependencies": { + "@parcel/codeframe": "2.3.1", + "@parcel/diagnostic": "2.3.1", + "@parcel/hash": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/markdown-ansi": "2.3.1", + "@parcel/source-map": "^2.0.0", + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/@parcel/workers": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/workers/-/workers-2.3.1.tgz", + "integrity": "sha512-e2P/9p5AYBLfNRs8n+57ChGrn5171oHwY54dz/jj0CrXKN1q0b+rNwzYsPaAtOicBoqmm1s5I3cjfO6GfJP65A==", + "dependencies": { + "@parcel/diagnostic": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "chrome-trace-event": "^1.0.2", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.3.1" + } + }, + "node_modules/@gatsbyjs/parcel-namer-relative-to-cwd/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/@gatsbyjs/potrace": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@gatsbyjs/potrace/-/potrace-2.2.0.tgz", + "integrity": "sha512-/RiLVFJA+CIYhceb6XL1kD1GZ5E2JBX38pld0fiGNiNwLl+Bb7TYZR72aQvcs3v+NOrSjbagUiCnIHYmEW4F7w==", + "dependencies": { + "jimp": "^0.16.1" + } + }, + "node_modules/@gatsbyjs/reach-router": { + "version": "1.3.7", + "resolved": "https://registry.npmjs.org/@gatsbyjs/reach-router/-/reach-router-1.3.7.tgz", + "integrity": "sha512-KQ5FvMb4BZUlSo+yQgd4t4WB8vkVPWfKjTpSl+Bx/FZhU6OL4lpwgfX7fXAY/18DogqyJCFiNAjV5eo3rQ5Alw==", + "dependencies": { + "invariant": "^2.2.3", + "prop-types": "^15.6.1", + "react-lifecycles-compat": "^3.0.4" + }, + "peerDependencies": { + "react": "15.x || 16.x || 17.x || 18.x", + "react-dom": "15.x || 16.x || 17.x || 18.x" + } + }, + "node_modules/@gatsbyjs/webpack-hot-middleware": { + "version": "2.25.3", + "resolved": "https://registry.npmjs.org/@gatsbyjs/webpack-hot-middleware/-/webpack-hot-middleware-2.25.3.tgz", + "integrity": "sha512-ul17OZ8Dlw+ATRbnuU+kwxuAlq9lKbYz/2uBS1FLCdgoPTF1H2heP7HbUbgfMZbfRQNcCG2rMscMnr32ritCDw==", + "dependencies": { + "ansi-html-community": "0.0.8", + "html-entities": "^2.3.3", + "strip-ansi": "^6.0.0" + } + }, + "node_modules/@graphql-codegen/add": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/add/-/add-3.1.1.tgz", + "integrity": "sha512-XkVwcqosa0CVBlL1HaQT0gp+EUfhuQE3LzrEpzMQLwchxaj/NPVYtOJL6MUHaYDsHzLqxWrufjfbeB3y2NQgRw==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.3.2", + "tslib": "~2.3.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/core": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/core/-/core-2.5.1.tgz", + "integrity": "sha512-alctBVl2hMnBXDLwkgmnFPrZVIiBDsWJSmxJcM4GKg1PB23+xuov35GE47YAyAhQItE1B1fbYnbb1PtGiDZ4LA==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.4.1", + "@graphql-tools/schema": "^8.1.2", + "@graphql-tools/utils": "^8.1.1", + "tslib": "~2.3.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/plugin-helpers": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/@graphql-codegen/plugin-helpers/-/plugin-helpers-2.4.2.tgz", + "integrity": "sha512-LJNvwAPv/sKtI3RnRDm+nPD+JeOfOuSOS4FFIpQCMUCyMnFcchV/CPTTv7tT12fLUpEg6XjuFfDBvOwndti30Q==", + "dependencies": { + "@graphql-tools/utils": "^8.5.2", + "change-case-all": "1.0.14", + "common-tags": "1.8.2", + "import-from": "4.0.0", + "lodash": "~4.17.0", + "tslib": "~2.3.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/schema-ast": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/schema-ast/-/schema-ast-2.4.1.tgz", + "integrity": "sha512-bIWlKk/ShoVJfghA4Rt1OWnd34/dQmZM/vAe6fu6QKyOh44aAdqPtYQ2dbTyFXoknmu504etKJGEDllYNUJRfg==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.3.2", + "@graphql-tools/utils": "^8.1.1", + "tslib": "~2.3.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/typescript": { + "version": "2.4.11", + "resolved": "https://registry.npmjs.org/@graphql-codegen/typescript/-/typescript-2.4.11.tgz", + "integrity": "sha512-K3oDLPJRH9Wgpg9TOvb7L+xrJZ8HxkIzV2umqGn54c+8DQjvnRFBIYRO0THgUBMnEauE2sEy6RZkGHGfgQUruA==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-codegen/schema-ast": "^2.4.1", + "@graphql-codegen/visitor-plugin-common": "2.8.0", + "auto-bind": "~4.0.0", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/typescript-operations": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/@graphql-codegen/typescript-operations/-/typescript-operations-2.4.0.tgz", + "integrity": "sha512-vJ15FLyWchuO2Xkp6uz7jJOdChiay7P9KJKFDILx/JTwjinU1fFa7iOvyeTvslqiUPxgsXthR5izdY+E5IyLkQ==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-codegen/typescript": "^2.4.11", + "@graphql-codegen/visitor-plugin-common": "2.8.0", + "auto-bind": "~4.0.0", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/typescript-operations/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-codegen/typescript/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-codegen/visitor-plugin-common": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/@graphql-codegen/visitor-plugin-common/-/visitor-plugin-common-2.8.0.tgz", + "integrity": "sha512-29MOaxBog7qaEhmeCzJn2mONSbcA+slCTzHN4nJ3aZl4KrC9V32rXlQpG5x0qHbFQ1LaG1f5gPO83xbiAeMBIw==", + "dependencies": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-tools/optimize": "^1.0.1", + "@graphql-tools/relay-operation-optimizer": "^6.3.7", + "@graphql-tools/utils": "^8.3.0", + "auto-bind": "~4.0.0", + "change-case-all": "1.0.14", + "dependency-graph": "^0.11.0", + "graphql-tag": "^2.11.0", + "parse-filepath": "^1.0.2", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^0.8.0 || ^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-codegen/visitor-plugin-common/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/batch-execute": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/@graphql-tools/batch-execute/-/batch-execute-7.1.2.tgz", + "integrity": "sha512-IuR2SB2MnC2ztA/XeTMTfWcA0Wy7ZH5u+nDkDNLAdX+AaSyDnsQS35sCmHqG0VOGTl7rzoyBWLCKGwSJplgtwg==", + "dependencies": { + "@graphql-tools/utils": "^7.7.0", + "dataloader": "2.0.0", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/batch-execute/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/batch-execute/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/batch-execute/node_modules/value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==", + "engines": { + "node": ">=12" + } + }, + "node_modules/@graphql-tools/code-file-loader": { + "version": "7.2.15", + "resolved": "https://registry.npmjs.org/@graphql-tools/code-file-loader/-/code-file-loader-7.2.15.tgz", + "integrity": "sha512-tqViC0MEXadcGofqDU4mL7eA/+TyM4iCP+XRkL/mdAMYWp1JdzMYUJcCH47ZK1+EvNctYx8sJOpSGlvLR4ZFFg==", + "dependencies": { + "@graphql-tools/graphql-tag-pluck": "7.2.7", + "@graphql-tools/utils": "8.6.10", + "globby": "^11.0.3", + "tslib": "~2.4.0", + "unixify": "^1.0.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/code-file-loader/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/delegate": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/delegate/-/delegate-7.1.5.tgz", + "integrity": "sha512-bQu+hDd37e+FZ0CQGEEczmRSfQRnnXeUxI/0miDV+NV/zCbEdIJj5tYFNrKT03W6wgdqx8U06d8L23LxvGri/g==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "@graphql-tools/batch-execute": "^7.1.2", + "@graphql-tools/schema": "^7.1.5", + "@graphql-tools/utils": "^7.7.1", + "dataloader": "2.0.0", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/delegate/node_modules/@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "dependencies": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/delegate/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/delegate/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/delegate/node_modules/value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==", + "engines": { + "node": ">=12" + } + }, + "node_modules/@graphql-tools/graphql-file-loader": { + "version": "6.2.7", + "resolved": "https://registry.npmjs.org/@graphql-tools/graphql-file-loader/-/graphql-file-loader-6.2.7.tgz", + "integrity": "sha512-5k2SNz0W87tDcymhEMZMkd6/vs6QawDyjQXWtqkuLTBF3vxjxPD1I4dwHoxgWPIjjANhXybvulD7E+St/7s9TQ==", + "dependencies": { + "@graphql-tools/import": "^6.2.6", + "@graphql-tools/utils": "^7.0.0", + "tslib": "~2.1.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/graphql-file-loader/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/graphql-file-loader/node_modules/@graphql-tools/utils/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/graphql-file-loader/node_modules/tslib": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.1.0.tgz", + "integrity": "sha512-hcVC3wYEziELGGmEEXue7D75zbwIIVUMWAVbHItGPx0ziyXxrOMQx4rQEVEV45Ut/1IotuEvwqPopzIOkDMf0A==" + }, + "node_modules/@graphql-tools/graphql-tag-pluck": { + "version": "7.2.7", + "resolved": "https://registry.npmjs.org/@graphql-tools/graphql-tag-pluck/-/graphql-tag-pluck-7.2.7.tgz", + "integrity": "sha512-leh6rVHNoGXjmbOtTaKNXQtFw4Gu7PokTlEOweNKYMssbH6L1zrKA0G4cE55s60JBo9SF4hjg8X7Gkech5t+mQ==", + "dependencies": { + "@babel/parser": "^7.16.8", + "@babel/traverse": "^7.16.8", + "@babel/types": "^7.16.8", + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/graphql-tag-pluck/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/import": { + "version": "6.6.14", + "resolved": "https://registry.npmjs.org/@graphql-tools/import/-/import-6.6.14.tgz", + "integrity": "sha512-XN6swtMdUxd9czmdNIK6yJ0w5t4FOUWSoSkYP0+to8j44r8zdM3nsAppoA0OLmsUY+JnTBgkW3jGlOFvqC3HWg==", + "dependencies": { + "@graphql-tools/utils": "8.6.10", + "resolve-from": "5.0.0", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/import/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/json-file-loader": { + "version": "6.2.6", + "resolved": "https://registry.npmjs.org/@graphql-tools/json-file-loader/-/json-file-loader-6.2.6.tgz", + "integrity": "sha512-CnfwBSY5926zyb6fkDBHnlTblHnHI4hoBALFYXnrg0Ev4yWU8B04DZl/pBRUc459VNgO2x8/mxGIZj2hPJG1EA==", + "dependencies": { + "@graphql-tools/utils": "^7.0.0", + "tslib": "~2.0.1" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/json-file-loader/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/json-file-loader/node_modules/@graphql-tools/utils/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/json-file-loader/node_modules/tslib": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.0.3.tgz", + "integrity": "sha512-uZtkfKblCEQtZKBF6EBXVZeQNl82yqtDQdv+eck8u7tdPxjLu2/lp5/uPW+um2tpuxINHWy3GhiccY7QgEaVHQ==" + }, + "node_modules/@graphql-tools/load": { + "version": "7.5.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/load/-/load-7.5.11.tgz", + "integrity": "sha512-a8sD3iHfxcbIwP0nSxF+DUAVg+/MuLNOizVJHcZGGS8AdDoezUsnWRkNDT6FlVqRoxHNbkpq8+6B55JKtqHSxg==", + "dependencies": { + "@graphql-tools/schema": "8.3.11", + "@graphql-tools/utils": "8.6.10", + "p-limit": "3.1.0", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/load/node_modules/p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dependencies": { + "yocto-queue": "^0.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/@graphql-tools/load/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/merge": { + "version": "8.2.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/merge/-/merge-8.2.11.tgz", + "integrity": "sha512-fsjJVdsk9GV1jj1Ed2AKLlHYlsf0ZadTK8X5KxFRE1ZSnKqh56BLVX93JrtOIAnsiHkwOK2TC43HGhApF1swpQ==", + "dependencies": { + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/merge/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/optimize": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/optimize/-/optimize-1.2.0.tgz", + "integrity": "sha512-l0PTqgHeorQdeOizUor6RB49eOAng9+abSxiC5/aHRo6hMmXVaqv5eqndlmxCpx9BkgNb3URQbK+ZZHVktkP/g==", + "dependencies": { + "tslib": "~2.3.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/relay-operation-optimizer": { + "version": "6.4.10", + "resolved": "https://registry.npmjs.org/@graphql-tools/relay-operation-optimizer/-/relay-operation-optimizer-6.4.10.tgz", + "integrity": "sha512-a5wDdXP7MmwZDy9R8+RZ0ajJBWX1Lk9sIG6uSIo5G/LnGpXncgBhKpJf5r6rOf0zsFLWnAkYm/dCDMpFaGE/Yw==", + "dependencies": { + "@graphql-tools/utils": "8.6.10", + "relay-compiler": "12.0.0", + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/relay-operation-optimizer/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/schema": { + "version": "8.3.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-8.3.11.tgz", + "integrity": "sha512-esMEnbyXbp8B5VEI4o395+x0G7Qmz3JSX5onFBF8HeLYcqWJasY5vBuWkO18VxrZpEnvnryodP6Y00bVag9O3Q==", + "dependencies": { + "@graphql-tools/merge": "8.2.11", + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0", + "value-or-promise": "1.0.11" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/schema/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/url-loader": { + "version": "6.10.1", + "resolved": "https://registry.npmjs.org/@graphql-tools/url-loader/-/url-loader-6.10.1.tgz", + "integrity": "sha512-DSDrbhQIv7fheQ60pfDpGD256ixUQIR6Hhf9Z5bRjVkXOCvO5XrkwoWLiU7iHL81GB1r0Ba31bf+sl+D4nyyfw==", + "dependencies": { + "@graphql-tools/delegate": "^7.0.1", + "@graphql-tools/utils": "^7.9.0", + "@graphql-tools/wrap": "^7.0.4", + "@microsoft/fetch-event-source": "2.0.1", + "@types/websocket": "1.0.2", + "abort-controller": "3.0.0", + "cross-fetch": "3.1.4", + "extract-files": "9.0.0", + "form-data": "4.0.0", + "graphql-ws": "^4.4.1", + "is-promise": "4.0.0", + "isomorphic-ws": "4.0.1", + "lodash": "4.17.21", + "meros": "1.1.4", + "subscriptions-transport-ws": "^0.9.18", + "sync-fetch": "0.3.0", + "tslib": "~2.2.0", + "valid-url": "1.0.9", + "ws": "7.4.5" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/url-loader/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/url-loader/node_modules/cross-fetch": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/cross-fetch/-/cross-fetch-3.1.4.tgz", + "integrity": "sha512-1eAtFWdIubi6T4XPy6ei9iUFoKpUkIF971QLN8lIvvvwueI65+Nw5haMNKUwfJxabqlIIDODJKGrQ66gxC0PbQ==", + "dependencies": { + "node-fetch": "2.6.1" + } + }, + "node_modules/@graphql-tools/url-loader/node_modules/node-fetch": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.1.tgz", + "integrity": "sha512-V4aYg89jEoVRxRb2fJdAg8FHvI7cEyYdVAh94HH0UIK8oJxUfkjlDQN9RbMx+bEjP7+ggMiFRprSti032Oipxw==", + "engines": { + "node": "4.x || >=6.0.0" + } + }, + "node_modules/@graphql-tools/url-loader/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/utils": { + "version": "8.6.10", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-8.6.10.tgz", + "integrity": "sha512-bJH9qwuyM3BP0PTU6/lvBDkk6jdEIOn+dbyk4pHMVNnvbJ1gZQwo62To8SHxxaUTus8OMhhVPSh9ApWXREURcg==", + "dependencies": { + "tslib": "~2.4.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/@graphql-tools/utils/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@graphql-tools/wrap": { + "version": "7.0.8", + "resolved": "https://registry.npmjs.org/@graphql-tools/wrap/-/wrap-7.0.8.tgz", + "integrity": "sha512-1NDUymworsOlb53Qfh7fonDi2STvqCtbeE68ntKY9K/Ju/be2ZNxrFSbrBHwnxWcN9PjISNnLcAyJ1L5tCUyhg==", + "dependencies": { + "@graphql-tools/delegate": "^7.1.5", + "@graphql-tools/schema": "^7.1.5", + "@graphql-tools/utils": "^7.8.1", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/wrap/node_modules/@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "dependencies": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/wrap/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/@graphql-tools/wrap/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/@graphql-tools/wrap/node_modules/value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==", + "engines": { + "node": ">=12" + } + }, + "node_modules/@hapi/address": { + "version": "2.1.4", + "resolved": "https://registry.npmjs.org/@hapi/address/-/address-2.1.4.tgz", + "integrity": "sha512-QD1PhQk+s31P1ixsX0H0Suoupp3VMXzIVMSwobR3F3MSUO2YCV0B7xqLcUw/Bh8yuvd3LhpyqLQWTNcRmp6IdQ==", + "deprecated": "Moved to 'npm install @sideway/address'" + }, + "node_modules/@hapi/bourne": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/@hapi/bourne/-/bourne-1.3.2.tgz", + "integrity": "sha512-1dVNHT76Uu5N3eJNTYcvxee+jzX4Z9lfciqRRHCU27ihbUcYi+iSc2iml5Ke1LXe1SyJCLA0+14Jh4tXJgOppA==", + "deprecated": "This version has been deprecated and is no longer supported or maintained" + }, + "node_modules/@hapi/hoek": { + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-9.3.0.tgz", + "integrity": "sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ==" + }, + "node_modules/@hapi/joi": { + "version": "15.1.1", + "resolved": "https://registry.npmjs.org/@hapi/joi/-/joi-15.1.1.tgz", + "integrity": "sha512-entf8ZMOK8sc+8YfeOlM8pCfg3b5+WZIKBfUaaJT8UsjAAPjartzxIYm3TIbjvA4u+u++KbcXD38k682nVHDAQ==", + "deprecated": "Switch to 'npm install joi'", + "dependencies": { + "@hapi/address": "2.x.x", + "@hapi/bourne": "1.x.x", + "@hapi/hoek": "8.x.x", + "@hapi/topo": "3.x.x" + } + }, + "node_modules/@hapi/joi/node_modules/@hapi/hoek": { + "version": "8.5.1", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-8.5.1.tgz", + "integrity": "sha512-yN7kbciD87WzLGc5539Tn0sApjyiGHAJgKvG9W8C7O+6c7qmoQMfVs0W4bX17eqz6C78QJqqFrtgdK5EWf6Qow==", + "deprecated": "This version has been deprecated and is no longer supported or maintained" + }, + "node_modules/@hapi/joi/node_modules/@hapi/topo": { + "version": "3.1.6", + "resolved": "https://registry.npmjs.org/@hapi/topo/-/topo-3.1.6.tgz", + "integrity": "sha512-tAag0jEcjwH+P2quUfipd7liWCNX2F8NvYjQp2wtInsZxnMlypdw0FtAOLxtvvkO+GSRRbmNi8m/5y42PQJYCQ==", + "deprecated": "This version has been deprecated and is no longer supported or maintained", + "dependencies": { + "@hapi/hoek": "^8.3.0" + } + }, + "node_modules/@hapi/topo": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/@hapi/topo/-/topo-5.1.0.tgz", + "integrity": "sha512-foQZKJig7Ob0BMAYBfcJk8d77QtOe7Wo4ox7ff1lQYoNNAb6jwcY1ncdoy2e9wQZzvNy7ODZCYJkK8kzmcAnAg==", + "dependencies": { + "@hapi/hoek": "^9.0.0" + } + }, + "node_modules/@humanwhocodes/config-array": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.5.0.tgz", + "integrity": "sha512-FagtKFz74XrTl7y6HCzQpwDfXP0yhxe9lHLD1UZxjvZIcbyRz8zTFF/yYNfSfzU414eDwZ1SrO0Qvtyf+wFMQg==", + "dependencies": { + "@humanwhocodes/object-schema": "^1.2.0", + "debug": "^4.1.1", + "minimatch": "^3.0.4" + }, + "engines": { + "node": ">=10.10.0" + } + }, + "node_modules/@humanwhocodes/config-array/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@humanwhocodes/object-schema": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz", + "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==" + }, + "node_modules/@iarna/toml": { + "version": "2.2.5", + "resolved": "https://registry.npmjs.org/@iarna/toml/-/toml-2.2.5.tgz", + "integrity": "sha512-trnsAYxU3xnS1gPHPyU961coFyLkh4gAD/0zQ5mymY4yOZ+CYvsPqUbOFSw0aDM4y0tV7tiFxL/1XfXPNC6IPg==" + }, + "node_modules/@icon-park/react": { + "version": "1.3.5", + "resolved": "https://registry.npmmirror.com/@icon-park/react/-/react-1.3.5.tgz", + "integrity": "sha512-FL3+BcYfFbRRl6DA6V2yA/QEoRRrhaDvMXLbFz0uW9lIsstQB6au/8aQxthB1G+eiqLxcw6yVULbmbBySn4Paw==", + "engines": { + "node": ">= 8.0.0", + "npm": ">= 5.0.0" + }, + "peerDependencies": { + "react": ">=16.9", + "react-dom": ">=16.9" + } + }, + "node_modules/@jimp/bmp": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/bmp/-/bmp-0.16.1.tgz", + "integrity": "sha512-iwyNYQeBawrdg/f24x3pQ5rEx+/GwjZcCXd3Kgc+ZUd+Ivia7sIqBsOnDaMZdKCBPlfW364ekexnlOqyVa0NWg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "bmp-js": "^0.1.0" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/core": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/core/-/core-0.16.1.tgz", + "integrity": "sha512-la7kQia31V6kQ4q1kI/uLimu8FXx7imWVajDGtwUG8fzePLWDFJyZl0fdIXVCL1JW2nBcRHidUot6jvlRDi2+g==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "any-base": "^1.1.0", + "buffer": "^5.2.0", + "exif-parser": "^0.1.12", + "file-type": "^9.0.0", + "load-bmfont": "^1.3.1", + "mkdirp": "^0.5.1", + "phin": "^2.9.1", + "pixelmatch": "^4.0.2", + "tinycolor2": "^1.4.1" + } + }, + "node_modules/@jimp/core/node_modules/file-type": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/file-type/-/file-type-9.0.0.tgz", + "integrity": "sha512-Qe/5NJrgIOlwijpq3B7BEpzPFcgzggOTagZmkXQY4LA6bsXKTUstK7Wp12lEJ/mLKTpvIZxmIuRcLYWT6ov9lw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/@jimp/custom": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/custom/-/custom-0.16.1.tgz", + "integrity": "sha512-DNUAHNSiUI/j9hmbatD6WN/EBIyeq4AO0frl5ETtt51VN1SvE4t4v83ZA/V6ikxEf3hxLju4tQ5Pc3zmZkN/3A==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/core": "^0.16.1" + } + }, + "node_modules/@jimp/gif": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/gif/-/gif-0.16.1.tgz", + "integrity": "sha512-r/1+GzIW1D5zrP4tNrfW+3y4vqD935WBXSc8X/wm23QTY9aJO9Lw6PEdzpYCEY+SOklIFKaJYUAq/Nvgm/9ryw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "gifwrap": "^0.9.2", + "omggif": "^1.0.9" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/jpeg": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/jpeg/-/jpeg-0.16.1.tgz", + "integrity": "sha512-8352zrdlCCLFdZ/J+JjBslDvml+fS3Z8gttdml0We759PnnZGqrnPRhkOEOJbNUlE+dD4ckLeIe6NPxlS/7U+w==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "jpeg-js": "0.4.2" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-blit": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-blit/-/plugin-blit-0.16.1.tgz", + "integrity": "sha512-fKFNARm32RoLSokJ8WZXHHH2CGzz6ire2n1Jh6u+XQLhk9TweT1DcLHIXwQMh8oR12KgjbgsMGvrMVlVknmOAg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-blur": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-blur/-/plugin-blur-0.16.1.tgz", + "integrity": "sha512-1WhuLGGj9MypFKRcPvmW45ht7nXkOKu+lg3n2VBzIB7r4kKNVchuI59bXaCYQumOLEqVK7JdB4glaDAbCQCLyw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-circle": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-circle/-/plugin-circle-0.16.1.tgz", + "integrity": "sha512-JK7yi1CIU7/XL8hdahjcbGA3V7c+F+Iw+mhMQhLEi7Q0tCnZ69YJBTamMiNg3fWPVfMuvWJJKOBRVpwNTuaZRg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-color": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-color/-/plugin-color-0.16.1.tgz", + "integrity": "sha512-9yQttBAO5SEFj7S6nJK54f+1BnuBG4c28q+iyzm1JjtnehjqMg6Ljw4gCSDCvoCQ3jBSYHN66pmwTV74SU1B7A==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "tinycolor2": "^1.4.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-contain": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-contain/-/plugin-contain-0.16.1.tgz", + "integrity": "sha512-44F3dUIjBDHN+Ym/vEfg+jtjMjAqd2uw9nssN67/n4FdpuZUVs7E7wadKY1RRNuJO+WgcD5aDQcsvurXMETQTg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-blit": ">=0.3.5", + "@jimp/plugin-resize": ">=0.3.5", + "@jimp/plugin-scale": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-cover": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-cover/-/plugin-cover-0.16.1.tgz", + "integrity": "sha512-YztWCIldBAVo0zxcQXR+a/uk3/TtYnpKU2CanOPJ7baIuDlWPsG+YE4xTsswZZc12H9Kl7CiziEbDtvF9kwA/Q==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-crop": ">=0.3.5", + "@jimp/plugin-resize": ">=0.3.5", + "@jimp/plugin-scale": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-crop": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-crop/-/plugin-crop-0.16.1.tgz", + "integrity": "sha512-UQdva9oQzCVadkyo3T5Tv2CUZbf0klm2cD4cWMlASuTOYgaGaFHhT9st+kmfvXjKL8q3STkBu/zUPV6PbuV3ew==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-displace": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-displace/-/plugin-displace-0.16.1.tgz", + "integrity": "sha512-iVAWuz2+G6Heu8gVZksUz+4hQYpR4R0R/RtBzpWEl8ItBe7O6QjORAkhxzg+WdYLL2A/Yd4ekTpvK0/qW8hTVw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-dither": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-dither/-/plugin-dither-0.16.1.tgz", + "integrity": "sha512-tADKVd+HDC9EhJRUDwMvzBXPz4GLoU6s5P7xkVq46tskExYSptgj5713J5Thj3NMgH9Rsqu22jNg1H/7tr3V9Q==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-fisheye": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-fisheye/-/plugin-fisheye-0.16.1.tgz", + "integrity": "sha512-BWHnc5hVobviTyIRHhIy9VxI1ACf4CeSuCfURB6JZm87YuyvgQh5aX5UDKtOz/3haMHXBLP61ZBxlNpMD8CG4A==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-flip": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-flip/-/plugin-flip-0.16.1.tgz", + "integrity": "sha512-KdxTf0zErfZ8DyHkImDTnQBuHby+a5YFdoKI/G3GpBl3qxLBvC+PWkS2F/iN3H7wszP7/TKxTEvWL927pypT0w==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-rotate": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-gaussian": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-gaussian/-/plugin-gaussian-0.16.1.tgz", + "integrity": "sha512-u9n4wjskh3N1mSqketbL6tVcLU2S5TEaFPR40K6TDv4phPLZALi1Of7reUmYpVm8mBDHt1I6kGhuCJiWvzfGyg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-invert": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-invert/-/plugin-invert-0.16.1.tgz", + "integrity": "sha512-2DKuyVXANH8WDpW9NG+PYFbehzJfweZszFYyxcaewaPLN0GxvxVLOGOPP1NuUTcHkOdMFbE0nHDuB7f+sYF/2w==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-mask": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-mask/-/plugin-mask-0.16.1.tgz", + "integrity": "sha512-snfiqHlVuj4bSFS0v96vo2PpqCDMe4JB+O++sMo5jF5mvGcGL6AIeLo8cYqPNpdO6BZpBJ8MY5El0Veckhr39Q==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-normalize": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-normalize/-/plugin-normalize-0.16.1.tgz", + "integrity": "sha512-dOQfIOvGLKDKXPU8xXWzaUeB0nvkosHw6Xg1WhS1Z5Q0PazByhaxOQkSKgUryNN/H+X7UdbDvlyh/yHf3ITRaw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-print": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-print/-/plugin-print-0.16.1.tgz", + "integrity": "sha512-ceWgYN40jbN4cWRxixym+csyVymvrryuKBQ+zoIvN5iE6OyS+2d7Mn4zlNgumSczb9GGyZZESIgVcBDA1ezq0Q==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "load-bmfont": "^1.4.0" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-blit": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-resize": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-resize/-/plugin-resize-0.16.1.tgz", + "integrity": "sha512-u4JBLdRI7dargC04p2Ha24kofQBk3vhaf0q8FwSYgnCRwxfvh2RxvhJZk9H7Q91JZp6wgjz/SjvEAYjGCEgAwQ==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-rotate": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-rotate/-/plugin-rotate-0.16.1.tgz", + "integrity": "sha512-ZUU415gDQ0VjYutmVgAYYxC9Og9ixu2jAGMCU54mSMfuIlmohYfwARQmI7h4QB84M76c9hVLdONWjuo+rip/zg==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-blit": ">=0.3.5", + "@jimp/plugin-crop": ">=0.3.5", + "@jimp/plugin-resize": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-scale": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-scale/-/plugin-scale-0.16.1.tgz", + "integrity": "sha512-jM2QlgThIDIc4rcyughD5O7sOYezxdafg/2Xtd1csfK3z6fba3asxDwthqPZAgitrLgiKBDp6XfzC07Y/CefUw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-resize": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-shadow": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-shadow/-/plugin-shadow-0.16.1.tgz", + "integrity": "sha512-MeD2Is17oKzXLnsphAa1sDstTu6nxscugxAEk3ji0GV1FohCvpHBcec0nAq6/czg4WzqfDts+fcPfC79qWmqrA==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-blur": ">=0.3.5", + "@jimp/plugin-resize": ">=0.3.5" + } + }, + "node_modules/@jimp/plugin-threshold": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-threshold/-/plugin-threshold-0.16.1.tgz", + "integrity": "sha512-iGW8U/wiCSR0+6syrPioVGoSzQFt4Z91SsCRbgNKTAk7D+XQv6OI78jvvYg4o0c2FOlwGhqz147HZV5utoSLxA==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5", + "@jimp/plugin-color": ">=0.8.0", + "@jimp/plugin-resize": ">=0.8.0" + } + }, + "node_modules/@jimp/plugins": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugins/-/plugins-0.16.1.tgz", + "integrity": "sha512-c+lCqa25b+4q6mJZSetlxhMoYuiltyS+ValLzdwK/47+aYsq+kcJNl+TuxIEKf59yr9+5rkbpsPkZHLF/V7FFA==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/plugin-blit": "^0.16.1", + "@jimp/plugin-blur": "^0.16.1", + "@jimp/plugin-circle": "^0.16.1", + "@jimp/plugin-color": "^0.16.1", + "@jimp/plugin-contain": "^0.16.1", + "@jimp/plugin-cover": "^0.16.1", + "@jimp/plugin-crop": "^0.16.1", + "@jimp/plugin-displace": "^0.16.1", + "@jimp/plugin-dither": "^0.16.1", + "@jimp/plugin-fisheye": "^0.16.1", + "@jimp/plugin-flip": "^0.16.1", + "@jimp/plugin-gaussian": "^0.16.1", + "@jimp/plugin-invert": "^0.16.1", + "@jimp/plugin-mask": "^0.16.1", + "@jimp/plugin-normalize": "^0.16.1", + "@jimp/plugin-print": "^0.16.1", + "@jimp/plugin-resize": "^0.16.1", + "@jimp/plugin-rotate": "^0.16.1", + "@jimp/plugin-scale": "^0.16.1", + "@jimp/plugin-shadow": "^0.16.1", + "@jimp/plugin-threshold": "^0.16.1", + "timm": "^1.6.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/png": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/png/-/png-0.16.1.tgz", + "integrity": "sha512-iyWoCxEBTW0OUWWn6SveD4LePW89kO7ZOy5sCfYeDM/oTPLpR8iMIGvZpZUz1b8kvzFr27vPst4E5rJhGjwsdw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "pngjs": "^3.3.3" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/tiff": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/tiff/-/tiff-0.16.1.tgz", + "integrity": "sha512-3K3+xpJS79RmSkAvFMgqY5dhSB+/sxhwTFA9f4AVHUK0oKW+u6r52Z1L0tMXHnpbAdR9EJ+xaAl2D4x19XShkQ==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "utif": "^2.0.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/types": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/types/-/types-0.16.1.tgz", + "integrity": "sha512-g1w/+NfWqiVW4CaXSJyD28JQqZtm2eyKMWPhBBDCJN9nLCN12/Az0WFF3JUAktzdsEC2KRN2AqB1a2oMZBNgSQ==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/bmp": "^0.16.1", + "@jimp/gif": "^0.16.1", + "@jimp/jpeg": "^0.16.1", + "@jimp/png": "^0.16.1", + "@jimp/tiff": "^0.16.1", + "timm": "^1.6.1" + }, + "peerDependencies": { + "@jimp/custom": ">=0.3.5" + } + }, + "node_modules/@jimp/utils": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/utils/-/utils-0.16.1.tgz", + "integrity": "sha512-8fULQjB0x4LzUSiSYG6ZtQl355sZjxbv8r9PPAuYHzS9sGiSHJQavNqK/nKnpDsVkU88/vRGcE7t3nMU0dEnVw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "regenerator-runtime": "^0.13.3" + } + }, + "node_modules/@jridgewell/gen-mapping": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.1.1.tgz", + "integrity": "sha512-sQXCasFk+U8lWYEe66WxRDOE9PjVz4vSM51fTu3Hw+ClTpUSQb718772vH3pyS5pShp6lvQM7SxgIDXXXmOX7w==", + "dependencies": { + "@jridgewell/set-array": "^1.0.0", + "@jridgewell/sourcemap-codec": "^1.4.10" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/resolve-uri": { + "version": "3.0.7", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.0.7.tgz", + "integrity": "sha512-8cXDaBBHOr2pQ7j77Y6Vp5VDT2sIqWyWQ56TjEq4ih/a4iST3dItRe8Q9fp0rrIl9DoKhWQtUQz/YpOxLkXbNA==", + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/set-array": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.1.tgz", + "integrity": "sha512-Ct5MqZkLGEXTVmQYbGtx9SVqD2fqwvdubdps5D3djjAkgkKwT918VNOz65pEHFaYTeWcukmJmH5SwsA9Tn2ObQ==", + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/sourcemap-codec": { + "version": "1.4.13", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.13.tgz", + "integrity": "sha512-GryiOJmNcWbovBxTfZSF71V/mXbgcV3MewDe3kIMCLyIh5e7SKAeUZs+rMnJ8jkMolZ/4/VsdBmMrw3l+VdZ3w==" + }, + "node_modules/@jridgewell/trace-mapping": { + "version": "0.3.11", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.11.tgz", + "integrity": "sha512-RllI476aSMsxzeI9TtlSMoNTgHDxEmnl6GkkHwhr0vdL8W+0WuesyI8Vd3rBOfrwtPXbPxdT9ADJdiOKgzxPQA==", + "dependencies": { + "@jridgewell/resolve-uri": "^3.0.3", + "@jridgewell/sourcemap-codec": "^1.4.10" + } + }, + "node_modules/@lezer/common": { + "version": "0.15.12", + "resolved": "https://registry.npmjs.org/@lezer/common/-/common-0.15.12.tgz", + "integrity": "sha512-edfwCxNLnzq5pBA/yaIhwJ3U3Kz8VAUOTRg0hhxaizaI1N+qxV7EXDv/kLCkLeq2RzSFvxexlaj5Mzfn2kY0Ig==" + }, + "node_modules/@lezer/lr": { + "version": "0.15.8", + "resolved": "https://registry.npmjs.org/@lezer/lr/-/lr-0.15.8.tgz", + "integrity": "sha512-bM6oE6VQZ6hIFxDNKk8bKPa14hqFrV07J/vHGOeiAbJReIaQXmkVb6xQu4MR+JBTLa5arGRyAAjJe1qaQt3Uvg==", + "dependencies": { + "@lezer/common": "^0.15.0" + } + }, + "node_modules/@mantine/core": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/core/-/core-4.2.5.tgz", + "integrity": "sha512-A/mixxpmCsA9acYVrCAurHrj/ikyLGksPqFILPJJpjC4uUDm85HHqVUa209VVLAKlvr26tX6CzDVqUeSe6s3Zg==", + "dependencies": { + "@mantine/styles": "4.2.5", + "@popperjs/core": "^2.9.3", + "@radix-ui/react-scroll-area": "^0.1.1", + "react-popper": "^2.2.5", + "react-textarea-autosize": "^8.3.2" + }, + "peerDependencies": { + "@mantine/hooks": "4.2.5", + "react": ">=16.8.0", + "react-dom": ">=16.8.0" + } + }, + "node_modules/@mantine/hooks": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/hooks/-/hooks-4.2.5.tgz", + "integrity": "sha512-CEpdUXPAC28rXosgo/Wxvs3ch9qC+QYfqh4AFkOH0+EKdlXkD1xRN8vv6pd5AHJtRZvGf7CZDuGKSjWgssvgeA==", + "peerDependencies": { + "react": ">=16.8.0" + } + }, + "node_modules/@mantine/ssr": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/ssr/-/ssr-4.2.5.tgz", + "integrity": "sha512-3Nt5PjCSKylAeyg1j6yct5hoX45J0jvxCwblphnhcYL+YVdNdaI2KL5Dg+KE/wcIAzOATquidUCFSHUwqcUVkg==", + "dependencies": { + "@emotion/cache": "11.7.1", + "@emotion/react": "11.7.1", + "@emotion/serialize": "1.0.2", + "@emotion/server": "11.4.0", + "@emotion/utils": "1.0.0", + "@mantine/styles": "4.2.5", + "csstype": "3.0.9", + "html-react-parser": "1.3.0" + }, + "peerDependencies": { + "react": ">=16.8.0", + "react-dom": ">=16.8.0" + } + }, + "node_modules/@mantine/ssr/node_modules/csstype": { + "version": "3.0.9", + "resolved": "https://registry.npmmirror.com/csstype/-/csstype-3.0.9.tgz", + "integrity": "sha512-rpw6JPxK6Rfg1zLOYCSwle2GFOOsnjmDYDaBwEcwoOg4qlsIVCN789VkBZDJAGi4T07gI4YSutR43t9Zz4Lzuw==" + }, + "node_modules/@mantine/styles": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/styles/-/styles-4.2.5.tgz", + "integrity": "sha512-A6sIm3+Aa4ZqpaIqpmiaCmkAJI4ow9vwmCgpvuwWYCspBhWTWQWtdpnNyWfnTjszYY1uSnx9mb50JlIsglgLNQ==", + "dependencies": { + "@emotion/cache": "11.7.1", + "@emotion/react": "11.7.1", + "@emotion/serialize": "1.0.2", + "@emotion/utils": "1.0.0", + "clsx": "^1.1.1", + "csstype": "3.0.9" + }, + "peerDependencies": { + "react": ">=16.8.0", + "react-dom": ">=16.8.0" + } + }, + "node_modules/@mantine/styles/node_modules/csstype": { + "version": "3.0.9", + "resolved": "https://registry.npmmirror.com/csstype/-/csstype-3.0.9.tgz", + "integrity": "sha512-rpw6JPxK6Rfg1zLOYCSwle2GFOOsnjmDYDaBwEcwoOg4qlsIVCN789VkBZDJAGi4T07gI4YSutR43t9Zz4Lzuw==" + }, + "node_modules/@microsoft/fetch-event-source": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/@microsoft/fetch-event-source/-/fetch-event-source-2.0.1.tgz", + "integrity": "sha512-W6CLUJ2eBMw3Rec70qrsEW0jOm/3twwJv21mrmj2yORiaVmVYGS4sSS5yUwvQc1ZlDLYGPnClVWmUUMagKNsfA==" + }, + "node_modules/@mischnic/json-sourcemap": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/@mischnic/json-sourcemap/-/json-sourcemap-0.1.0.tgz", + "integrity": "sha512-dQb3QnfNqmQNYA4nFSN/uLaByIic58gOXq4Y4XqLOWmOrw73KmJPt/HLyG0wvn1bnR6mBKs/Uwvkh+Hns1T0XA==", + "dependencies": { + "@lezer/common": "^0.15.7", + "@lezer/lr": "^0.15.4", + "json5": "^2.2.1" + }, + "engines": { + "node": ">=12.0.0" + } + }, + "node_modules/@nodelib/fs.scandir": { + "version": "2.1.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", + "integrity": "sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g==", + "dependencies": { + "@nodelib/fs.stat": "2.0.5", + "run-parallel": "^1.1.9" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/@nodelib/fs.stat": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz", + "integrity": "sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A==", + "engines": { + "node": ">= 8" + } + }, + "node_modules/@nodelib/fs.walk": { + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz", + "integrity": "sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg==", + "dependencies": { + "@nodelib/fs.scandir": "2.1.5", + "fastq": "^1.6.0" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/@parcel/bundler-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/bundler-default/-/bundler-default-2.5.0.tgz", + "integrity": "sha512-7CJzE17SirCXjcRgBcnqWO/5EOA1raq/3OIKtT4cxbjpDQGHZpjpEEZiMNRpEpdNMxDSlsG8mAkXTYGL2VVWRw==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/cache": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/cache/-/cache-2.5.0.tgz", + "integrity": "sha512-3kOO3cZQv0FAKhrMHGLdb4Qtzpmy78Q6jPN3u8eCY4yqeDTnyQBZvWNHoyCm5WlmL8y6Q6REYMbETLxSH1ggAQ==", + "dependencies": { + "@parcel/fs": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/utils": "2.5.0", + "lmdb": "2.2.4" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.5.0" + } + }, + "node_modules/@parcel/cache/node_modules/lmdb": { + "version": "2.2.4", + "resolved": "https://registry.npmjs.org/lmdb/-/lmdb-2.2.4.tgz", + "integrity": "sha512-gto+BB2uEob8qRiTlOq+R3uX0YNHsX9mjxj9Sbdue/LIKqu6IlZjrsjKeGyOMquc/474GEqFyX2pdytpydp0rQ==", + "hasInstallScript": true, + "dependencies": { + "msgpackr": "^1.5.4", + "nan": "^2.14.2", + "node-gyp-build": "^4.2.3", + "ordered-binary": "^1.2.4", + "weak-lru-cache": "^1.2.2" + } + }, + "node_modules/@parcel/codeframe": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/codeframe/-/codeframe-2.5.0.tgz", + "integrity": "sha512-qafqL8Vu2kr932cCWESoDEEoAeKVi7/xdzTBuhzEJng1AfmRT0rCbt/P4ao3RjiDyozPSjXsHOqM6GDZcto4eQ==", + "dependencies": { + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/compressor-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/compressor-raw/-/compressor-raw-2.5.0.tgz", + "integrity": "sha512-I5Zs+2f1ue4sTPdfT8BNsLfTZl48sMWLk2Io3elUJjH/SS9kO7ut5ChkuJtt77ZS35m0OF+ZCt3ICTJdnDG8eA==", + "dependencies": { + "@parcel/plugin": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/core": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/core/-/core-2.5.0.tgz", + "integrity": "sha512-dygDmPsfAYJKTnUftcbEzjCik7AAaPbFvJW8ETYz8diyjkAG9y6hvCAZIrJE5pNOjFzg32en4v4UWv8Sqlzl9g==", + "dependencies": { + "@mischnic/json-sourcemap": "^0.1.0", + "@parcel/cache": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/events": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/graph": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/package-manager": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "abortcontroller-polyfill": "^1.1.9", + "base-x": "^3.0.8", + "browserslist": "^4.6.6", + "clone": "^2.1.1", + "dotenv": "^7.0.0", + "dotenv-expand": "^5.1.0", + "json5": "^2.2.0", + "msgpackr": "^1.5.4", + "nullthrows": "^1.1.1", + "semver": "^5.7.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/core/node_modules/dotenv": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-7.0.0.tgz", + "integrity": "sha512-M3NhsLbV1i6HuGzBUH8vXrtxOk+tWmzWKDMbAVSUp3Zsjm7ywFeuwrUXhmhQyRK1q5B5GGy7hcXPbj3bnfZg2g==", + "engines": { + "node": ">=6" + } + }, + "node_modules/@parcel/core/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/@parcel/diagnostic": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/diagnostic/-/diagnostic-2.5.0.tgz", + "integrity": "sha512-KiMGGRpEV7wl5gjcxBKcgX84a+cG+IEn94gwy5LK3lENR09nuKShqqgKGAmj/17CobJgw1QNP94/H4Md+oxIWg==", + "dependencies": { + "@mischnic/json-sourcemap": "^0.1.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/events": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/events/-/events-2.5.0.tgz", + "integrity": "sha512-Gc2LPwL1H34Ony5MENbKZg7wvCscZ4x9y7Fu92sfbdWpLo3K13hVtsX3TMIIgYt3B7R7OmO8yR880U2T+JfVkQ==", + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/fs": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/fs/-/fs-2.5.0.tgz", + "integrity": "sha512-YYr14BWtx/bJ+hu6PPQQ6G/3omOTWgVqEw+UFI3iQH3P6+e0LRXW/Ja1yAcJeepGcTwIP0opnXZBQOm8PBQ2SA==", + "dependencies": { + "@parcel/fs-search": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/watcher": "^2.0.0", + "@parcel/workers": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.5.0" + } + }, + "node_modules/@parcel/fs-search": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/fs-search/-/fs-search-2.5.0.tgz", + "integrity": "sha512-uBONkz9ZCNSOqbPGWJY3MNl+pqBTfvzHH9+4UhzHEHPArvK2oD0+syYPVE60+zGrxybXTESYMCJp4bHvH6Z2hA==", + "dependencies": { + "detect-libc": "^1.0.3" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/graph": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/graph/-/graph-2.5.0.tgz", + "integrity": "sha512-qa2VtG08dJyTaWrxYAkMIlkoDRSPoiqLDNxxHKplkcxAjXBUw0/AkWaz82VO5r1G6jfOj+nM30ajH9uygZYwbw==", + "dependencies": { + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/hash": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/hash/-/hash-2.5.0.tgz", + "integrity": "sha512-47JL0XpB7UvIW6Ijf8vv+yVMt9dLvB/lRlBHFmAkmovisueVMVbYD7smxVZnCSehD8UH8BcymKbMzyL5dimgoQ==", + "dependencies": { + "detect-libc": "^1.0.3", + "xxhash-wasm": "^0.4.2" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/logger": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/logger/-/logger-2.5.0.tgz", + "integrity": "sha512-pT1L3ceH6trL1N3I3r2HawPjz/PCubOo/Kazu7IeXsMsKVjj1a6AeieZHzkNZIbhiGPtm/cHbBNLz2zTWDLeOA==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/events": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/markdown-ansi": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/markdown-ansi/-/markdown-ansi-2.5.0.tgz", + "integrity": "sha512-ixkNF3KWIqxMlfxTe9Gb2cp/uNmklQev8VEUxujMVxmUfGyQs4859zdJIQlIinabWYhArhsXATkVf3MzCUN6TQ==", + "dependencies": { + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/namer-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/namer-default/-/namer-default-2.5.0.tgz", + "integrity": "sha512-ahGQqHJzsWE5Qux8zXMAU+lyNBOl+ZpcOFzRGE2DWOsmAlytsHl7DBVCQvzUyNBFg1/HmIj+7D4efv2kjR7rTg==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/node-resolver-core": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/node-resolver-core/-/node-resolver-core-2.5.0.tgz", + "integrity": "sha512-XQvpguiIwQcu75cscLDFOVhjsjuPzXbuMaaZ7XxxUEl0PscIgu/GfKYxTfTruN3cRl+CaQH6qBAMfjLaFng6lQ==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/optimizer-terser": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/optimizer-terser/-/optimizer-terser-2.5.0.tgz", + "integrity": "sha512-PZ3UHBGfjE49/Jloopsd38Hxg4qzsrdepWP53mCuVP7Aw605Y4QtYuB1ho3VV0oXfKQVq+uI7lVIBsuW4K6vqA==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1", + "terser": "^5.2.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/package-manager": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/package-manager/-/package-manager-2.5.0.tgz", + "integrity": "sha512-zTuF55/lITUjw9dUU/X0HiF++589xbPXw/zUiG9T6s8BQThLvrxAhYP89S719pw7cTqDimGkTxnIuK+a0djEkg==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "semver": "^5.7.1" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.5.0" + } + }, + "node_modules/@parcel/package-manager/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/@parcel/packager-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/packager-js/-/packager-js-2.5.0.tgz", + "integrity": "sha512-aJAKOTgXdxO3V9O7+2DCVOtne128WwXmUAOVThnMRo7f3zMVSAR7Mxc9pEsuTzPfj8UBXgFBRfdJUSCgsMxiSw==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "globals": "^13.2.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/packager-js/node_modules/globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "dependencies": { + "type-fest": "^0.20.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@parcel/packager-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/packager-raw/-/packager-raw-2.5.0.tgz", + "integrity": "sha512-aHV0oogeiqxhxS1lsttw15EvG3DDWK3FV7+F+7hoaAy+xg89K56NTp6j43Jtw9iyU1/HnZRGBE2hF3C7N73oKw==", + "dependencies": { + "@parcel/plugin": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/plugin": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/plugin/-/plugin-2.5.0.tgz", + "integrity": "sha512-obtb6/Gql6YFQ86bdv75A2Noabx8679reFZeyfKKf0L7Lppx4DFQetXwM9XVy7Gx6hJ1Ekm3UMuuIyVJk33YHQ==", + "dependencies": { + "@parcel/types": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/reporter-dev-server": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/reporter-dev-server/-/reporter-dev-server-2.5.0.tgz", + "integrity": "sha512-wvxAiW42AxJ3B8jtvowJcP4/cTV8zY48SfKg61YKYu1yUO+TtyJIjHQzDW2XuT34cIGFY97Gr0i+AVu44RyUuQ==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/resolver-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/resolver-default/-/resolver-default-2.5.0.tgz", + "integrity": "sha512-39PkZpVr/+iYS11u+lA84vIsKm/yisltTVmUjlYsDnExiuV1c8OSbSdYZ3JMx+7CYPE0bWbosX2AGilIwIMWpQ==", + "dependencies": { + "@parcel/node-resolver-core": "2.5.0", + "@parcel/plugin": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/runtime-browser-hmr": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-browser-hmr/-/runtime-browser-hmr-2.5.0.tgz", + "integrity": "sha512-oPAo8Zf06gXCpt41nyvK7kv2HH1RrHAGgOqttyjStwAFlm5MZKs7BgtJzO58LfJN8g3sMY0cNdG17fB/4f8q6Q==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/runtime-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-js/-/runtime-js-2.5.0.tgz", + "integrity": "sha512-gPC2PbNAiooULP71wF5twe4raekuXsR1Hw/ahITDoqsZdXHzG3CkoCjYL3CkmBGiKQgMMocCyN1E2oBzAH8Kyw==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/runtime-react-refresh": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-react-refresh/-/runtime-react-refresh-2.5.0.tgz", + "integrity": "sha512-+8RuDKFdFYIQTrXG4MRhG9XqkkYEHn0zxKyOJ/IkDDfSEhY0na+EyhrneFUwIvDX63gLPkxceXAg0gwBqXPK/Q==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "react-refresh": "^0.9.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/runtime-service-worker": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-service-worker/-/runtime-service-worker-2.5.0.tgz", + "integrity": "sha512-STuDlU0fPXeWpAmbayY7o04F0eHy6FTOFeT5KQ0PTxtdEa3Ey8QInP/NVE52Yv0aVQtesWukGrNEFCERlkbFRw==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/source-map": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/@parcel/source-map/-/source-map-2.0.2.tgz", + "integrity": "sha512-NnUrPYLpYB6qyx2v6bcRPn/gVigmGG6M6xL8wIg/i0dP1GLkuY1nf+Hqdf63FzPTqqT7K3k6eE5yHPQVMO5jcA==", + "dependencies": { + "detect-libc": "^1.0.3" + }, + "engines": { + "node": "^12.18.3 || >=14" + } + }, + "node_modules/@parcel/transformer-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-js/-/transformer-js-2.5.0.tgz", + "integrity": "sha512-Cp8Ic+Au3OcskCRZszmo47z3bqcZ7rfPv2xZYXpXY2TzEc3IV0bKje57bZektoY8LW9LkYM9iBO/WhkVoT6LIg==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "@swc/helpers": "^0.3.6", + "browserslist": "^4.6.6", + "detect-libc": "^1.0.3", + "nullthrows": "^1.1.1", + "regenerator-runtime": "^0.13.7", + "semver": "^5.7.1" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + }, + "peerDependencies": { + "@parcel/core": "^2.5.0" + } + }, + "node_modules/@parcel/transformer-js/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/@parcel/transformer-json": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-json/-/transformer-json-2.5.0.tgz", + "integrity": "sha512-661sByA7TkR6Lmxt+hqV4h2SAt+7lgc58DzmUYArpEl1fQnMuQuaB0kQeHzi6fDD2+2G6o7EC+DuwBZKa479TA==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "json5": "^2.2.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/transformer-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-raw/-/transformer-raw-2.5.0.tgz", + "integrity": "sha512-I3zjE1u9+Wj90Qqs1V2FTm6iC6SAyOVUthwVZkZey+qbQG/ok682Ez2XjLu7MyQCo9BJNwF/nfOa1hHr3MaJEQ==", + "dependencies": { + "@parcel/plugin": "2.5.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/transformer-react-refresh-wrap": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-react-refresh-wrap/-/transformer-react-refresh-wrap-2.5.0.tgz", + "integrity": "sha512-VPqVBxhTN4OQwcjsdyxrv+smjAm4s6dbSWAplgPwdOITMv+a0tjhhJU37WnRC+xxTrbEqRcOt96JvGOkPb8i7g==", + "dependencies": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "react-refresh": "^0.9.0" + }, + "engines": { + "node": ">= 12.0.0", + "parcel": "^2.5.0" + } + }, + "node_modules/@parcel/types": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/types/-/types-2.5.0.tgz", + "integrity": "sha512-bA0fhG6aXSGYEVo5Dt96x6lseUQHeVZVzgmiRdZsvb614Gvx22ItfaKhPmAVbM9vzbObZDHl9l9G2Ovw8Xve4g==", + "dependencies": { + "@parcel/cache": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/package-manager": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/workers": "2.5.0", + "utility-types": "^3.10.0" + } + }, + "node_modules/@parcel/utils": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/utils/-/utils-2.5.0.tgz", + "integrity": "sha512-kaLGXtQuOOH55KZqXdYDvczhh3mk2eeTVqrrXuuihGjbLKYFlUW2tFDm+5r2s9nCPwTQxOO43ZEOCKSnia+e4w==", + "dependencies": { + "@parcel/codeframe": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/markdown-ansi": "2.5.0", + "@parcel/source-map": "^2.0.0", + "chalk": "^4.1.0" + }, + "engines": { + "node": ">= 12.0.0" + } + }, + "node_modules/@parcel/watcher": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@parcel/watcher/-/watcher-2.0.5.tgz", + "integrity": "sha512-x0hUbjv891omnkcHD7ZOhiyyUqUUR6MNjq89JhEI3BxppeKWAm6NPQsqqRrAkCJBogdT/o/My21sXtTI9rJIsw==", + "hasInstallScript": true, + "dependencies": { + "node-addon-api": "^3.2.1", + "node-gyp-build": "^4.3.0" + }, + "engines": { + "node": ">= 10.0.0" + } + }, + "node_modules/@parcel/workers": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/workers/-/workers-2.5.0.tgz", + "integrity": "sha512-/Ow5OKJWs+9OzV3Jy4J++VnbNx0j3ls/M1CGVBLiBWyCada9DMtquYoBQ4Sk6Uam50BKkIFYetGOeXPNQyyMjg==", + "dependencies": { + "@parcel/diagnostic": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "chrome-trace-event": "^1.0.2", + "nullthrows": "^1.1.1" + }, + "engines": { + "node": ">= 12.0.0" + }, + "peerDependencies": { + "@parcel/core": "^2.5.0" + } + }, + "node_modules/@pmmmwh/react-refresh-webpack-plugin": { + "version": "0.4.3", + "resolved": "https://registry.npmjs.org/@pmmmwh/react-refresh-webpack-plugin/-/react-refresh-webpack-plugin-0.4.3.tgz", + "integrity": "sha512-br5Qwvh8D2OQqSXpd1g/xqXKnK0r+Jz6qVKBbWmpUcrbGOxUrf39V5oZ1876084CGn18uMdR5uvPqBv9UqtBjQ==", + "dependencies": { + "ansi-html": "^0.0.7", + "error-stack-parser": "^2.0.6", + "html-entities": "^1.2.1", + "native-url": "^0.2.6", + "schema-utils": "^2.6.5", + "source-map": "^0.7.3" + }, + "engines": { + "node": ">= 10.x" + }, + "peerDependencies": { + "@types/webpack": "4.x", + "react-refresh": ">=0.8.3 <0.10.0", + "sockjs-client": "^1.4.0", + "type-fest": "^0.13.1", + "webpack": ">=4.43.0 <6.0.0", + "webpack-dev-server": "3.x", + "webpack-hot-middleware": "2.x", + "webpack-plugin-serve": "0.x || 1.x" + }, + "peerDependenciesMeta": { + "@types/webpack": { + "optional": true + }, + "sockjs-client": { + "optional": true + }, + "type-fest": { + "optional": true + }, + "webpack-dev-server": { + "optional": true + }, + "webpack-hot-middleware": { + "optional": true + }, + "webpack-plugin-serve": { + "optional": true + } + } + }, + "node_modules/@pmmmwh/react-refresh-webpack-plugin/node_modules/ansi-html": { + "version": "0.0.7", + "resolved": "https://registry.npmjs.org/ansi-html/-/ansi-html-0.0.7.tgz", + "integrity": "sha1-gTWEAhliqenm/QOflA0S9WynhZ4=", + "engines": [ + "node >= 0.8.0" + ], + "bin": { + "ansi-html": "bin/ansi-html" + } + }, + "node_modules/@pmmmwh/react-refresh-webpack-plugin/node_modules/html-entities": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/html-entities/-/html-entities-1.4.0.tgz", + "integrity": "sha512-8nxjcBcd8wovbeKx7h3wTji4e6+rhaVuPNpMqwWgnHh+N9ToqsCs6XztWRBPQ+UtzsoMAdKZtUENoVzU/EMtZA==" + }, + "node_modules/@popperjs/core": { + "version": "2.11.5", + "resolved": "https://registry.npmmirror.com/@popperjs/core/-/core-2.11.5.tgz", + "integrity": "sha512-9X2obfABZuDVLCgPK9aX0a/x4jaOEweTTWE2+9sr0Qqqevj2Uv5XorvusThmc9XGYpS9yI+fhh8RTafBtGposw==" + }, + "node_modules/@radix-ui/number": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/number/-/number-0.1.0.tgz", + "integrity": "sha512-rpf6QiOWLHAkM4FEMYu9i+5Jr8cKT893+R4mPpcdsy4LD7omr9JfdOqj/h/xPA5+EcVrpMMlU6rrRYpUB5UI8g==", + "dependencies": { + "@babel/runtime": "^7.13.10" + } + }, + "node_modules/@radix-ui/primitive": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/primitive/-/primitive-0.1.0.tgz", + "integrity": "sha512-tqxZKybwN5Fa3VzZry4G6mXAAb9aAqKmPtnVbZpL0vsBwvOHTBwsjHVPXylocYLwEtBY9SCe665bYnNB515uoA==", + "dependencies": { + "@babel/runtime": "^7.13.10" + } + }, + "node_modules/@radix-ui/react-compose-refs": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-compose-refs/-/react-compose-refs-0.1.0.tgz", + "integrity": "sha512-eyclbh+b77k+69Dk72q3694OHrn9B3QsoIRx7ywX341U9RK1ThgQjMFZoPtmZNQTksXHLNEiefR8hGVeFyInGg==", + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-context": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-context/-/react-context-0.1.1.tgz", + "integrity": "sha512-PkyVX1JsLBioeu0jB9WvRpDBBLtLZohVDT3BB5CTSJqActma8S8030P57mWZb4baZifMvN7KKWPAA40UmWKkQg==", + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-presence": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-presence/-/react-presence-0.1.2.tgz", + "integrity": "sha512-3BRlFZraooIUfRlyN+b/Xs5hq1lanOOo/+3h6Pwu2GMFjkGKKa4Rd51fcqGqnVlbr3jYg+WLuGyAV4KlgqwrQw==", + "dependencies": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-compose-refs": "0.1.0", + "@radix-ui/react-use-layout-effect": "0.1.0" + }, + "peerDependencies": { + "react": ">=16.8" + } + }, + "node_modules/@radix-ui/react-primitive": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-primitive/-/react-primitive-0.1.4.tgz", + "integrity": "sha512-6gSl2IidySupIMJFjYnDIkIWRyQdbu/AHK7rbICPani+LW4b0XdxBXc46og/iZvuwW8pjCS8I2SadIerv84xYA==", + "dependencies": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-slot": "0.1.2" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-scroll-area": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-scroll-area/-/react-scroll-area-0.1.4.tgz", + "integrity": "sha512-QHxRsjy+hsHwQYJ9cCNgSJ5+6ioZu1KhwD1UOXoHNciuFGMX08v+uJPKXIz+ySv03Rx6cOz6f/Fk5aPHRMFi/A==", + "dependencies": { + "@babel/runtime": "^7.13.10", + "@radix-ui/number": "0.1.0", + "@radix-ui/primitive": "0.1.0", + "@radix-ui/react-compose-refs": "0.1.0", + "@radix-ui/react-context": "0.1.1", + "@radix-ui/react-presence": "0.1.2", + "@radix-ui/react-primitive": "0.1.4", + "@radix-ui/react-use-callback-ref": "0.1.0", + "@radix-ui/react-use-direction": "0.1.0", + "@radix-ui/react-use-layout-effect": "0.1.0" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-slot": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-slot/-/react-slot-0.1.2.tgz", + "integrity": "sha512-ADkqfL+agEzEguU3yS26jfB50hRrwf7U4VTwAOZEmi/g+ITcBWe12yM46ueS/UCIMI9Py+gFUaAdxgxafFvY2Q==", + "dependencies": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-compose-refs": "0.1.0" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-use-callback-ref": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-0.1.0.tgz", + "integrity": "sha512-Va041McOFFl+aV+sejvl0BS2aeHx86ND9X/rVFmEFQKTXCp6xgUK0NGUAGcgBlIjnJSbMYPGEk1xKSSlVcN2Aw==", + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-use-direction": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-direction/-/react-use-direction-0.1.0.tgz", + "integrity": "sha512-NajpY/An9TCPSfOVkgWIdXJV+VuWl67PxB6kOKYmtNAFHvObzIoh8o0n9sAuwSAyFCZVq211FEf9gvVDRhOyiA==", + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@radix-ui/react-use-layout-effect": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-0.1.0.tgz", + "integrity": "sha512-+wdeS51Y+E1q1Wmd+1xSSbesZkpVj4jsg0BojCbopWvgq5iBvixw5vgemscdh58ep98BwUbsFYnrywFhV9yrVg==", + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "peerDependencies": { + "react": "^16.8 || ^17.0" + } + }, + "node_modules/@sideway/address": { + "version": "4.1.4", + "resolved": "https://registry.npmjs.org/@sideway/address/-/address-4.1.4.tgz", + "integrity": "sha512-7vwq+rOHVWjyXxVlR76Agnvhy8I9rpzjosTESvmhNeXOXdZZB15Fl+TI9x1SiHZH5Jv2wTGduSxFDIaq0m3DUw==", + "dependencies": { + "@hapi/hoek": "^9.0.0" + } + }, + "node_modules/@sideway/formula": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@sideway/formula/-/formula-3.0.0.tgz", + "integrity": "sha512-vHe7wZ4NOXVfkoRb8T5otiENVlT7a3IAiw7H5M2+GO+9CDgcVUUsX1zalAztCmwyOr2RUTGJdgB+ZvSVqmdHmg==" + }, + "node_modules/@sideway/pinpoint": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@sideway/pinpoint/-/pinpoint-2.0.0.tgz", + "integrity": "sha512-RNiOoTPkptFtSVzQevY/yWtZwf/RxyVnPy/OcA9HBM3MlGDnBEYL5B41H0MTn0Uec8Hi+2qUtTfG2WWZBmMejQ==" + }, + "node_modules/@sindresorhus/is": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-4.6.0.tgz", + "integrity": "sha512-t09vSN3MdfsyCHoFcTRCH/iUtG7OJ0CsjzB8cjAmKc/va/kIgeDI/TxsigdncE/4be734m0cvIYwNaV4i2XqAw==", + "engines": { + "node": ">=10" + } + }, + "node_modules/@sindresorhus/slugify": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@sindresorhus/slugify/-/slugify-1.1.2.tgz", + "integrity": "sha512-V9nR/W0Xd9TSGXpZ4iFUcFGhuOJtZX82Fzxj1YISlbSgKvIiNa7eLEZrT0vAraPOt++KHauIVNYgGRgjc13dXA==", + "dependencies": { + "@sindresorhus/transliterate": "^0.1.1", + "escape-string-regexp": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/@sindresorhus/slugify/node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/@sindresorhus/transliterate": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/@sindresorhus/transliterate/-/transliterate-0.1.2.tgz", + "integrity": "sha512-5/kmIOY9FF32nicXH+5yLNTX4NJ4atl7jRgqAJuIn/iyDFXBktOKDxCvyGE/EzmF4ngSUvjXxQUQlQiZ5lfw+w==", + "dependencies": { + "escape-string-regexp": "^2.0.0", + "lodash.deburr": "^4.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/@sindresorhus/transliterate/node_modules/escape-string-regexp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", + "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "engines": { + "node": ">=8" + } + }, + "node_modules/@swc/helpers": { + "version": "0.3.13", + "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.3.13.tgz", + "integrity": "sha512-A1wswJhnqaLRn8uYVQ8YiNTtY5i/JIPmV08EXXjjTresIkUVUEUaFv/wXVhGXfRNYMvHPkuoMR1Nb6NgpxGjNg==", + "dependencies": { + "tslib": "^2.4.0" + } + }, + "node_modules/@swc/helpers/node_modules/tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + }, + "node_modules/@szmarczak/http-timer": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-4.0.6.tgz", + "integrity": "sha512-4BAffykYOgO+5nzBWYwE3W90sBgLJoUPRWWcL8wlyiM8IB8ipJz3UMJ9KXQd1RKQXpKp8Tutn80HZtWsu2u76w==", + "dependencies": { + "defer-to-connect": "^2.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/@tokenizer/token": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/@tokenizer/token/-/token-0.3.0.tgz", + "integrity": "sha512-OvjF+z51L3ov0OyAU0duzsYuvO01PH7x4t6DJx+guahgTnBHkhJdG7soQeTSFLWN3efnHyibZ4Z8l2EuWwJN3A==" + }, + "node_modules/@trysound/sax": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/@trysound/sax/-/sax-0.2.0.tgz", + "integrity": "sha512-L7z9BgrNEcYyUYtF+HaEfiS5ebkh9jXqbszz7pC0hRBPaatV0XjSD3+eHrpqFemQfgwiFF0QPIarnIihIDn7OA==", + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/@turist/fetch": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@turist/fetch/-/fetch-7.2.0.tgz", + "integrity": "sha512-2x7EGw+6OJ29phunsbGvtxlNmSfcuPcyYudkMbi8gARCP9eJ1CtuMvnVUHL//O9Ixi9SJiug8wNt6lj86pN8XQ==", + "dependencies": { + "@types/node-fetch": "2" + }, + "peerDependencies": { + "node-fetch": "2" + } + }, + "node_modules/@turist/time": { + "version": "0.0.2", + "resolved": "https://registry.npmjs.org/@turist/time/-/time-0.0.2.tgz", + "integrity": "sha512-qLOvfmlG2vCVw5fo/oz8WAZYlpe5a5OurgTj3diIxJCdjRHpapC+vQCz3er9LV79Vcat+DifBjeAhOAdmndtDQ==" + }, + "node_modules/@types/cacheable-request": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/@types/cacheable-request/-/cacheable-request-6.0.2.tgz", + "integrity": "sha512-B3xVo+dlKM6nnKTcmm5ZtY/OL8bOAOd2Olee9M1zft65ox50OzjEHW91sDiU9j6cvW8Ejg1/Qkf4xd2kugApUA==", + "dependencies": { + "@types/http-cache-semantics": "*", + "@types/keyv": "*", + "@types/node": "*", + "@types/responselike": "*" + } + }, + "node_modules/@types/common-tags": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/@types/common-tags/-/common-tags-1.8.1.tgz", + "integrity": "sha512-20R/mDpKSPWdJs5TOpz3e7zqbeCNuMCPhV7Yndk9KU2Rbij2r5W4RzwDPkzC+2lzUqXYu9rFzTktCBnDjHuNQg==" + }, + "node_modules/@types/component-emitter": { + "version": "1.2.11", + "resolved": "https://registry.npmjs.org/@types/component-emitter/-/component-emitter-1.2.11.tgz", + "integrity": "sha512-SRXjM+tfsSlA9VuG8hGO2nft2p8zjXCK1VcC6N4NXbBbYbSia9kzCChYQajIjzIqOOOuh5Ock6MmV2oux4jDZQ==" + }, + "node_modules/@types/configstore": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/@types/configstore/-/configstore-2.1.1.tgz", + "integrity": "sha1-zR6FU2M60xhcPy8jns/10mQ+krY=" + }, + "node_modules/@types/cookie": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/@types/cookie/-/cookie-0.4.1.tgz", + "integrity": "sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q==" + }, + "node_modules/@types/cors": { + "version": "2.8.12", + "resolved": "https://registry.npmjs.org/@types/cors/-/cors-2.8.12.tgz", + "integrity": "sha512-vt+kDhq/M2ayberEtJcIN/hxXy1Pk+59g2FV/ZQceeaTyCtCucjL2Q7FXlFjtWn4n15KCr1NE2lNNFhp0lEThw==" + }, + "node_modules/@types/debug": { + "version": "0.0.30", + "resolved": "https://registry.npmjs.org/@types/debug/-/debug-0.0.30.tgz", + "integrity": "sha512-orGL5LXERPYsLov6CWs3Fh6203+dXzJkR7OnddIr2514Hsecwc8xRpzCapshBbKFImCsvS/mk6+FWiN5LyZJAQ==" + }, + "node_modules/@types/eslint": { + "version": "7.29.0", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-7.29.0.tgz", + "integrity": "sha512-VNcvioYDH8/FxaeTKkM4/TiTwt6pBV9E3OfGmvaw8tPl0rrHCJ4Ll15HRT+pMiFAf/MLQvAzC+6RzUMEL9Ceng==", + "dependencies": { + "@types/estree": "*", + "@types/json-schema": "*" + } + }, + "node_modules/@types/eslint-scope": { + "version": "3.7.3", + "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.3.tgz", + "integrity": "sha512-PB3ldyrcnAicT35TWPs5IcwKD8S333HMaa2VVv4+wdvebJkjWuW/xESoB8IwRcog8HYVYamb1g/R31Qv5Bx03g==", + "dependencies": { + "@types/eslint": "*", + "@types/estree": "*" + } + }, + "node_modules/@types/estree": { + "version": "0.0.51", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-0.0.51.tgz", + "integrity": "sha512-CuPgU6f3eT/XgKKPqKd/gLZV1Xmvf1a2R5POBOGQa6uv82xpls89HU5zKeVoyR8XzHd1RGNOlQlvUe3CFkjWNQ==" + }, + "node_modules/@types/get-port": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/@types/get-port/-/get-port-3.2.0.tgz", + "integrity": "sha512-TiNg8R1kjDde5Pub9F9vCwZA/BNW9HeXP5b9j7Qucqncy/McfPZ6xze/EyBdXS5FhMIGN6Fx3vg75l5KHy3V1Q==" + }, + "node_modules/@types/glob": { + "version": "5.0.37", + "resolved": "https://registry.npmjs.org/@types/glob/-/glob-5.0.37.tgz", + "integrity": "sha512-ATA/xrS7CZ3A2WCPVY4eKdNpybq56zqlTirnHhhyOztZM/lPxJzusOBI3BsaXbu6FrUluqzvMlI4sZ6BDYMlMg==", + "dependencies": { + "@types/minimatch": "*", + "@types/node": "*" + } + }, + "node_modules/@types/hast": { + "version": "2.3.4", + "resolved": "https://registry.npmmirror.com/@types/hast/-/hast-2.3.4.tgz", + "integrity": "sha512-wLEm0QvaoawEDoTRwzTXp4b4jpwiJDvR5KMnFnVodm3scufTlBOWRD6N1OBf9TZMhjlNsSfcO5V+7AF4+Vy+9g==", + "dependencies": { + "@types/unist": "*" + } + }, + "node_modules/@types/http-cache-semantics": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/@types/http-cache-semantics/-/http-cache-semantics-4.0.1.tgz", + "integrity": "sha512-SZs7ekbP8CN0txVG2xVRH6EgKmEm31BOxA07vkFaETzZz1xh+cbt8BcI0slpymvwhx5dlFnQG2rTlPVQn+iRPQ==" + }, + "node_modules/@types/http-proxy": { + "version": "1.17.9", + "resolved": "https://registry.npmjs.org/@types/http-proxy/-/http-proxy-1.17.9.tgz", + "integrity": "sha512-QsbSjA/fSk7xB+UXlCT3wHBy5ai9wOcNDWwZAtud+jXhwOM3l+EYZh8Lng4+/6n8uar0J7xILzqftJdJ/Wdfkw==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@types/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha512-3YP80IxxFJB4b5tYC2SUPwkg0XQLiu0nWvhRgEatgjf+29IcWO9X1k8xRv5DGssJ/lCrjYTjQPcobJr2yWIVuQ==" + }, + "node_modules/@types/json-schema": { + "version": "7.0.11", + "resolved": "https://registry.npmjs.org/@types/json-schema/-/json-schema-7.0.11.tgz", + "integrity": "sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ==" + }, + "node_modules/@types/json5": { + "version": "0.0.29", + "resolved": "https://registry.npmjs.org/@types/json5/-/json5-0.0.29.tgz", + "integrity": "sha1-7ihweulOEdK4J7y+UnC86n8+ce4=" + }, + "node_modules/@types/keyv": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@types/keyv/-/keyv-3.1.4.tgz", + "integrity": "sha512-BQ5aZNSCpj7D6K2ksrRCTmKRLEpnPvWDiLPfoGyhZ++8YtiK9d/3DBKPJgry359X/P1PfruyYwvnvwFjuEiEIg==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/lodash": { + "version": "4.14.182", + "resolved": "https://registry.npmjs.org/@types/lodash/-/lodash-4.14.182.tgz", + "integrity": "sha512-/THyiqyQAP9AfARo4pF+aCGcyiQ94tX/Is2I7HofNRqoYLgN1PBoOWu2/zTA5zMxzP5EFutMtWtGAFRKUe961Q==" + }, + "node_modules/@types/mdast": { + "version": "3.0.10", + "resolved": "https://registry.npmmirror.com/@types/mdast/-/mdast-3.0.10.tgz", + "integrity": "sha512-W864tg/Osz1+9f4lrGTZpCSO5/z4608eUp19tbozkq2HJK6i3z1kT0H9tlADXuYIb1YYOBByU4Jsqkk75q48qA==", + "dependencies": { + "@types/unist": "*" + } + }, + "node_modules/@types/minimatch": { + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/@types/minimatch/-/minimatch-3.0.5.tgz", + "integrity": "sha512-Klz949h02Gz2uZCMGwDUSDS1YBlTdDDgbWHi+81l29tQALUtvz4rAYi5uoVhE5Lagoq6DeqAUlbrHvW/mXDgdQ==" + }, + "node_modules/@types/minimist": { + "version": "1.2.2", + "resolved": "https://registry.npmmirror.com/@types/minimist/-/minimist-1.2.2.tgz", + "integrity": "sha512-jhuKLIRrhvCPLqwPcx6INqmKeiA5EWrsCOPhrlFSrbrmU4ZMPjj5Ul/oLCMDO98XRUIwVm78xICz4EPCektzeQ==" + }, + "node_modules/@types/mkdirp": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/@types/mkdirp/-/mkdirp-0.5.2.tgz", + "integrity": "sha512-U5icWpv7YnZYGsN4/cmh3WD2onMY0aJIiTE6+51TwJCttdHvtCYmkBNOobHlXwrJRL0nkH9jH4kD+1FAdMN4Tg==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/node": { + "version": "17.0.32", + "resolved": "https://registry.npmjs.org/@types/node/-/node-17.0.32.tgz", + "integrity": "sha512-eAIcfAvhf/BkHcf4pkLJ7ECpBAhh9kcxRBpip9cTiO+hf+aJrsxYxBeS6OXvOd9WqNAJmavXVpZvY1rBjNsXmw==" + }, + "node_modules/@types/node-fetch": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/@types/node-fetch/-/node-fetch-2.6.1.tgz", + "integrity": "sha512-oMqjURCaxoSIsHSr1E47QHzbmzNR5rK8McHuNb11BOM9cHcIK3Avy0s/b2JlXHoQGTYS3NsvWzV1M0iK7l0wbA==", + "dependencies": { + "@types/node": "*", + "form-data": "^3.0.0" + } + }, + "node_modules/@types/node-fetch/node_modules/form-data": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-3.0.1.tgz", + "integrity": "sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg==", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/@types/normalize-package-data": { + "version": "2.4.1", + "resolved": "https://registry.npmmirror.com/@types/normalize-package-data/-/normalize-package-data-2.4.1.tgz", + "integrity": "sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw==" + }, + "node_modules/@types/parse-json": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/@types/parse-json/-/parse-json-4.0.0.tgz", + "integrity": "sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA==" + }, + "node_modules/@types/parse5": { + "version": "5.0.3", + "resolved": "https://registry.npmmirror.com/@types/parse5/-/parse5-5.0.3.tgz", + "integrity": "sha512-kUNnecmtkunAoQ3CnjmMkzNU/gtxG8guhi+Fk2U/kOpIKjIMKnXGp4IJCgQJrXSgMsWYimYG4TGjz/UzbGEBTw==" + }, + "node_modules/@types/prop-types": { + "version": "15.7.5", + "resolved": "https://registry.npmjs.org/@types/prop-types/-/prop-types-15.7.5.tgz", + "integrity": "sha512-JCB8C6SnDoQf0cNycqd/35A7MjcnK+ZTqE7judS6o7utxUCg6imJg3QK2qzHKszlTjcj2cn+NwMB2i96ubpj7w==" + }, + "node_modules/@types/q": { + "version": "1.5.5", + "resolved": "https://registry.npmjs.org/@types/q/-/q-1.5.5.tgz", + "integrity": "sha512-L28j2FcJfSZOnL1WBjDYp2vUHCeIFlyYI/53EwD/rKUBQ7MtUUfbQWiyKJGpcnv4/WgrhWsFKrcPstcAt/J0tQ==" + }, + "node_modules/@types/reach__router": { + "version": "1.3.10", + "resolved": "https://registry.npmjs.org/@types/reach__router/-/reach__router-1.3.10.tgz", + "integrity": "sha512-iHAFGaVOrWi00/q7oBybggGsz5TOmwOW4M1H9sT7i9lly4qFC8XOgsdf6jUsoaOz2sknFHALEtZqCoDbokdJ2Q==", + "dependencies": { + "@types/react": "*" + } + }, + "node_modules/@types/react": { + "version": "18.0.9", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.0.9.tgz", + "integrity": "sha512-9bjbg1hJHUm4De19L1cHiW0Jvx3geel6Qczhjd0qY5VKVE2X5+x77YxAepuCwVh4vrgZJdgEJw48zrhRIeF4Nw==", + "dependencies": { + "@types/prop-types": "*", + "@types/scheduler": "*", + "csstype": "^3.0.2" + } + }, + "node_modules/@types/responselike": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/@types/responselike/-/responselike-1.0.0.tgz", + "integrity": "sha512-85Y2BjiufFzaMIlvJDvTTB8Fxl2xfLo4HgmHzVBz08w4wDePCTjYw66PdrolO0kzli3yam/YCgRufyo1DdQVTA==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/rimraf": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@types/rimraf/-/rimraf-2.0.5.tgz", + "integrity": "sha512-YyP+VfeaqAyFmXoTh3HChxOQMyjByRMsHU7kc5KOJkSlXudhMhQIALbYV7rHh/l8d2lX3VUQzprrcAgWdRuU8g==", + "dependencies": { + "@types/glob": "*", + "@types/node": "*" + } + }, + "node_modules/@types/scheduler": { + "version": "0.16.2", + "resolved": "https://registry.npmjs.org/@types/scheduler/-/scheduler-0.16.2.tgz", + "integrity": "sha512-hppQEBDmlwhFAXKJX2KnWLYu5yMfi91yazPb2l+lbJiwW+wdo1gNeRA+3RgNSO39WYX2euey41KEwnqesU2Jew==" + }, + "node_modules/@types/sharp": { + "version": "0.30.2", + "resolved": "https://registry.npmjs.org/@types/sharp/-/sharp-0.30.2.tgz", + "integrity": "sha512-uLCBwjDg/BTcQit0dpNGvkIjvH3wsb8zpaJePCjvONBBSfaKHoxXBIuq1MT8DMQEfk2fKYnpC9QExCgFhkGkMQ==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/@types/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha1-EHPEvIJHVK49EM+riKsCN7qWTk0=" + }, + "node_modules/@types/unist": { + "version": "2.0.6", + "resolved": "https://registry.npmmirror.com/@types/unist/-/unist-2.0.6.tgz", + "integrity": "sha512-PBjIUxZHOuj0R15/xuwJYjFi+KZdNFrehocChv4g5hu6aFroHue8m0lBP0POdK2nKzbw0cgV1mws8+V/JAcEkQ==" + }, + "node_modules/@types/websocket": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/@types/websocket/-/websocket-1.0.2.tgz", + "integrity": "sha512-B5m9aq7cbbD/5/jThEr33nUY8WEfVi6A2YKCTOvw5Ldy7mtsOkqRvGjnzy6g7iMMDsgu7xREuCzqATLDLQVKcQ==", + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/yoga-layout": { + "version": "1.9.2", + "resolved": "https://registry.npmjs.org/@types/yoga-layout/-/yoga-layout-1.9.2.tgz", + "integrity": "sha512-S9q47ByT2pPvD65IvrWp7qppVMpk9WGMbVq9wbWZOHg6tnXSD4vyhao6nOSBwwfDdV2p3Kx9evA9vI+XWTfDvw==" + }, + "node_modules/@typescript-eslint/eslint-plugin": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-4.33.0.tgz", + "integrity": "sha512-aINiAxGVdOl1eJyVjaWn/YcVAq4Gi/Yo35qHGCnqbWVz61g39D0h23veY/MA0rFFGfxK7TySg2uwDeNv+JgVpg==", + "dependencies": { + "@typescript-eslint/experimental-utils": "4.33.0", + "@typescript-eslint/scope-manager": "4.33.0", + "debug": "^4.3.1", + "functional-red-black-tree": "^1.0.1", + "ignore": "^5.1.8", + "regexpp": "^3.1.0", + "semver": "^7.3.5", + "tsutils": "^3.21.0" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependencies": { + "@typescript-eslint/parser": "^4.0.0", + "eslint": "^5.0.0 || ^6.0.0 || ^7.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/experimental-utils": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/experimental-utils/-/experimental-utils-4.33.0.tgz", + "integrity": "sha512-zeQjOoES5JFjTnAhI5QY7ZviczMzDptls15GFsI6jyUOq0kOf9+WonkhtlIhh0RgHRnqj5gdNxW5j1EvAyYg6Q==", + "dependencies": { + "@types/json-schema": "^7.0.7", + "@typescript-eslint/scope-manager": "4.33.0", + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/typescript-estree": "4.33.0", + "eslint-scope": "^5.1.1", + "eslint-utils": "^3.0.0" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependencies": { + "eslint": "*" + } + }, + "node_modules/@typescript-eslint/parser": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-4.33.0.tgz", + "integrity": "sha512-ZohdsbXadjGBSK0/r+d87X0SBmKzOq4/S5nzK6SBgJspFo9/CUDJ7hjayuze+JK7CZQLDMroqytp7pOcFKTxZA==", + "dependencies": { + "@typescript-eslint/scope-manager": "4.33.0", + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/typescript-estree": "4.33.0", + "debug": "^4.3.1" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependencies": { + "eslint": "^5.0.0 || ^6.0.0 || ^7.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/parser/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/scope-manager": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-4.33.0.tgz", + "integrity": "sha512-5IfJHpgTsTZuONKbODctL4kKuQje/bzBRkwHE8UOZ4f89Zeddg+EGZs8PD8NcN4LdM3ygHWYB3ukPAYjvl/qbQ==", + "dependencies": { + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/visitor-keys": "4.33.0" + }, + "engines": { + "node": "^8.10.0 || ^10.13.0 || >=11.10.1" + } + }, + "node_modules/@typescript-eslint/types": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-4.33.0.tgz", + "integrity": "sha512-zKp7CjQzLQImXEpLt2BUw1tvOMPfNoTAfb8l51evhYbOEEzdWyQNmHWWGPR6hwKJDAi+1VXSBmnhL9kyVTTOuQ==", + "engines": { + "node": "^8.10.0 || ^10.13.0 || >=11.10.1" + } + }, + "node_modules/@typescript-eslint/typescript-estree": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-4.33.0.tgz", + "integrity": "sha512-rkWRY1MPFzjwnEVHsxGemDzqqddw2QbTJlICPD9p9I9LfsO8fdmfQPOX3uKfUaGRDFJbfrtm/sXhVXN4E+bzCA==", + "dependencies": { + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/visitor-keys": "4.33.0", + "debug": "^4.3.1", + "globby": "^11.0.3", + "is-glob": "^4.0.1", + "semver": "^7.3.5", + "tsutils": "^3.21.0" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/typescript-estree/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/visitor-keys": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-4.33.0.tgz", + "integrity": "sha512-uqi/2aSz9g2ftcHWf8uLPJA70rUv6yuMW5Bohw+bwcuzaxQIHaKFZCKGoGXIrc9vkTJ3+0txM73K0Hq3d5wgIg==", + "dependencies": { + "@typescript-eslint/types": "4.33.0", + "eslint-visitor-keys": "^2.0.0" + }, + "engines": { + "node": "^8.10.0 || ^10.13.0 || >=11.10.1" + } + }, + "node_modules/@vercel/webpack-asset-relocator-loader": { + "version": "1.7.2", + "resolved": "https://registry.npmjs.org/@vercel/webpack-asset-relocator-loader/-/webpack-asset-relocator-loader-1.7.2.tgz", + "integrity": "sha512-pdMwUawmAtH/LScbjKJq/y2+gZFggFMc2tlJrlPSrgKajvYPEis3L9QKcMyC9RN1Xos4ezAP5AJfRCNN6RMKCQ==" + }, + "node_modules/@webassemblyjs/ast": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.1.tgz", + "integrity": "sha512-ukBh14qFLjxTQNTXocdyksN5QdM28S1CxHt2rdskFyL+xFV7VremuBLVbmCePj+URalXBENx/9Lm7lnhihtCSw==", + "dependencies": { + "@webassemblyjs/helper-numbers": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1" + } + }, + "node_modules/@webassemblyjs/floating-point-hex-parser": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.1.tgz", + "integrity": "sha512-iGRfyc5Bq+NnNuX8b5hwBrRjzf0ocrJPI6GWFodBFzmFnyvrQ83SHKhmilCU/8Jv67i4GJZBMhEzltxzcNagtQ==" + }, + "node_modules/@webassemblyjs/helper-api-error": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.1.tgz", + "integrity": "sha512-RlhS8CBCXfRUR/cwo2ho9bkheSXG0+NwooXcc3PAILALf2QLdFyj7KGsKRbVc95hZnhnERon4kW/D3SZpp6Tcg==" + }, + "node_modules/@webassemblyjs/helper-buffer": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.1.tgz", + "integrity": "sha512-gwikF65aDNeeXa8JxXa2BAk+REjSyhrNC9ZwdT0f8jc4dQQeDQ7G4m0f2QCLPJiMTTO6wfDmRmj/pW0PsUvIcA==" + }, + "node_modules/@webassemblyjs/helper-numbers": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.1.tgz", + "integrity": "sha512-vDkbxiB8zfnPdNK9Rajcey5C0w+QJugEglN0of+kmO8l7lDb77AnlKYQF7aarZuCrv+l0UvqL+68gSDr3k9LPQ==", + "dependencies": { + "@webassemblyjs/floating-point-hex-parser": "1.11.1", + "@webassemblyjs/helper-api-error": "1.11.1", + "@xtuc/long": "4.2.2" + } + }, + "node_modules/@webassemblyjs/helper-wasm-bytecode": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.1.tgz", + "integrity": "sha512-PvpoOGiJwXeTrSf/qfudJhwlvDQxFgelbMqtq52WWiXC6Xgg1IREdngmPN3bs4RoO83PnL/nFrxucXj1+BX62Q==" + }, + "node_modules/@webassemblyjs/helper-wasm-section": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.1.tgz", + "integrity": "sha512-10P9No29rYX1j7F3EVPX3JvGPQPae+AomuSTPiF9eBQeChHI6iqjMIwR9JmOJXwpnn/oVGDk7I5IlskuMwU/pg==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1" + } + }, + "node_modules/@webassemblyjs/ieee754": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ieee754/-/ieee754-1.11.1.tgz", + "integrity": "sha512-hJ87QIPtAMKbFq6CGTkZYJivEwZDbQUgYd3qKSadTNOhVY7p+gfP6Sr0lLRVTaG1JjFj+r3YchoqRYxNH3M0GQ==", + "dependencies": { + "@xtuc/ieee754": "^1.2.0" + } + }, + "node_modules/@webassemblyjs/leb128": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/leb128/-/leb128-1.11.1.tgz", + "integrity": "sha512-BJ2P0hNZ0u+Th1YZXJpzW6miwqQUGcIHT1G/sf72gLVD9DZ5AdYTqPNbHZh6K1M5VmKvFXwGSWZADz+qBWxeRw==", + "dependencies": { + "@xtuc/long": "4.2.2" + } + }, + "node_modules/@webassemblyjs/utf8": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/utf8/-/utf8-1.11.1.tgz", + "integrity": "sha512-9kqcxAEdMhiwQkHpkNiorZzqpGrodQQ2IGrHHxCy+Ozng0ofyMA0lTqiLkVs1uzTRejX+/O0EOT7KxqVPuXosQ==" + }, + "node_modules/@webassemblyjs/wasm-edit": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.1.tgz", + "integrity": "sha512-g+RsupUC1aTHfR8CDgnsVRVZFJqdkFHpsHMfJuWQzWU3tvnLC07UqHICfP+4XyL2tnr1amvl1Sdp06TnYCmVkA==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/helper-wasm-section": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1", + "@webassemblyjs/wasm-opt": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1", + "@webassemblyjs/wast-printer": "1.11.1" + } + }, + "node_modules/@webassemblyjs/wasm-gen": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.1.tgz", + "integrity": "sha512-F7QqKXwwNlMmsulj6+O7r4mmtAlCWfO/0HdgOxSklZfQcDu0TpLiD1mRt/zF25Bk59FIjEuGAIyn5ei4yMfLhA==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/ieee754": "1.11.1", + "@webassemblyjs/leb128": "1.11.1", + "@webassemblyjs/utf8": "1.11.1" + } + }, + "node_modules/@webassemblyjs/wasm-opt": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.1.tgz", + "integrity": "sha512-VqnkNqnZlU5EB64pp1l7hdm3hmQw7Vgqa0KF/KCNO9sIpI6Fk6brDEiX+iCOYrvMuBWDws0NkTOxYEb85XQHHw==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1" + } + }, + "node_modules/@webassemblyjs/wasm-parser": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.1.tgz", + "integrity": "sha512-rrBujw+dJu32gYB7/Lup6UhdkPx9S9SnobZzRVL7VcBH9Bt9bCBLEuX/YXOOtBsOZ4NQrRykKhffRWHvigQvOA==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-api-error": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/ieee754": "1.11.1", + "@webassemblyjs/leb128": "1.11.1", + "@webassemblyjs/utf8": "1.11.1" + } + }, + "node_modules/@webassemblyjs/wast-printer": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.1.tgz", + "integrity": "sha512-IQboUWM4eKzWW+N/jij2sRatKMh99QEelo3Eb2q0qXkvPRISAj8Qxtmw5itwqK+TTkBuUIE45AxYPToqPtL5gg==", + "dependencies": { + "@webassemblyjs/ast": "1.11.1", + "@xtuc/long": "4.2.2" + } + }, + "node_modules/@xtuc/ieee754": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@xtuc/ieee754/-/ieee754-1.2.0.tgz", + "integrity": "sha512-DX8nKgqcGwsc0eJSqYt5lwP4DH5FlHnmuWWBRy7X0NcaGR0ZtuyeESgMwTYVEtxmsNGY+qit4QYT/MIYTOTPeA==" + }, + "node_modules/@xtuc/long": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/@xtuc/long/-/long-4.2.2.tgz", + "integrity": "sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ==" + }, + "node_modules/abbrev": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/abbrev/-/abbrev-1.1.1.tgz", + "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" + }, + "node_modules/abort-controller": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", + "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", + "dependencies": { + "event-target-shim": "^5.0.0" + }, + "engines": { + "node": ">=6.5" + } + }, + "node_modules/abortcontroller-polyfill": { + "version": "1.7.3", + "resolved": "https://registry.npmjs.org/abortcontroller-polyfill/-/abortcontroller-polyfill-1.7.3.tgz", + "integrity": "sha512-zetDJxd89y3X99Kvo4qFx8GKlt6GsvN3UcRZHwU6iFA/0KiOmhkTVhe8oRoTBiTVPZu09x3vCra47+w8Yz1+2Q==" + }, + "node_modules/accepts": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/accepts/-/accepts-1.3.8.tgz", + "integrity": "sha512-PYAthTa2m2VKxuvSD3DPC/Gy+U+sOA1LAuT8mkmRuvw+NACSaeXEQ+NHcVF7rONl6qcaxV3Uuemwawk+7+SJLw==", + "dependencies": { + "mime-types": "~2.1.34", + "negotiator": "0.6.3" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/acorn": { + "version": "7.4.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-7.4.1.tgz", + "integrity": "sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A==", + "bin": { + "acorn": "bin/acorn" + }, + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/acorn-import-assertions": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.8.0.tgz", + "integrity": "sha512-m7VZ3jwz4eK6A4Vtt8Ew1/mNbP24u0FhdyfA7fSvnJR6LMdfOYnmuIrrJAgrYfYJ10F/otaHTtrtrtmHdMNzEw==", + "peerDependencies": { + "acorn": "^8" + } + }, + "node_modules/acorn-jsx": { + "version": "5.3.2", + "resolved": "https://registry.npmjs.org/acorn-jsx/-/acorn-jsx-5.3.2.tgz", + "integrity": "sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ==", + "peerDependencies": { + "acorn": "^6.0.0 || ^7.0.0 || ^8.0.0" + } + }, + "node_modules/address": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/address/-/address-1.1.2.tgz", + "integrity": "sha512-aT6camzM4xEA54YVJYSqxz1kv4IHnQZRtThJJHhUMRExaU5spC7jX5ugSwTaTgJliIgs4VhZOk7htClvQ/LmRA==", + "engines": { + "node": ">= 0.12.0" + } + }, + "node_modules/adjust-sourcemap-loader": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/adjust-sourcemap-loader/-/adjust-sourcemap-loader-3.0.0.tgz", + "integrity": "sha512-YBrGyT2/uVQ/c6Rr+t6ZJXniY03YtHGMJQYal368burRGYKqhx9qGTWqcBU5s1CwYY9E/ri63RYyG1IacMZtqw==", + "dependencies": { + "loader-utils": "^2.0.0", + "regex-parser": "^2.2.11" + }, + "engines": { + "node": ">=8.9" + } + }, + "node_modules/ajv": { + "version": "6.12.6", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.12.6.tgz", + "integrity": "sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==", + "dependencies": { + "fast-deep-equal": "^3.1.1", + "fast-json-stable-stringify": "^2.0.0", + "json-schema-traverse": "^0.4.1", + "uri-js": "^4.2.2" + } + }, + "node_modules/ajv-keywords": { + "version": "3.5.2", + "resolved": "https://registry.npmjs.org/ajv-keywords/-/ajv-keywords-3.5.2.tgz", + "integrity": "sha512-5p6WTN0DdTGVQk6VjcEju19IgaHudalcfabD7yhDGeA6bcQnmL+CpveLJq/3hvfwd1aof6L386Ougkx6RfyMIQ==", + "peerDependencies": { + "ajv": "^6.9.1" + } + }, + "node_modules/amdefine": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/amdefine/-/amdefine-1.0.1.tgz", + "integrity": "sha512-S2Hw0TtNkMJhIabBwIojKL9YHO5T0n5eNqWJ7Lrlel/zDbftQpxpapi8tZs3X1HWa+u+QeydGmzzNU0m09+Rcg==", + "engines": { + "node": ">=0.4.2" + } + }, + "node_modules/anser": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/anser/-/anser-2.1.1.tgz", + "integrity": "sha512-nqLm4HxOTpeLOxcmB3QWmV5TcDFhW9y/fyQ+hivtDFcK4OQ+pQ5fzPnXHM1Mfcm0VkLtvVi1TCPr++Qy0Q/3EQ==" + }, + "node_modules/ansi-align": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/ansi-align/-/ansi-align-3.0.1.tgz", + "integrity": "sha512-IOfwwBF5iczOjp/WeY4YxyjqAFMQoZufdQWDd19SEExbVLNXqvpzSJ/M7Za4/sCPmQ0+GRquoA7bGcINcxew6w==", + "dependencies": { + "string-width": "^4.1.0" + } + }, + "node_modules/ansi-colors": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", + "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "engines": { + "node": ">=6" + } + }, + "node_modules/ansi-escapes": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-3.2.0.tgz", + "integrity": "sha512-cBhpre4ma+U0T1oM5fXg7Dy1Jw7zzwv7lt/GoCpr+hDQJoYnKVPLL4dCvSEFMmQurOQvSrwT7SL/DAlhBI97RQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/ansi-html-community": { + "version": "0.0.8", + "resolved": "https://registry.npmjs.org/ansi-html-community/-/ansi-html-community-0.0.8.tgz", + "integrity": "sha512-1APHAyr3+PCamwNw3bXCPp4HFLONZt/yIH0sZp0/469KWNTEy+qN5jQ3GVX6DMZ1UXAi34yVwtTeaG/HpBuuzw==", + "engines": [ + "node >= 0.8.0" + ], + "bin": { + "ansi-html": "bin/ansi-html" + } + }, + "node_modules/ansi-regex": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", + "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dependencies": { + "color-convert": "^1.9.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/any-base": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/any-base/-/any-base-1.1.0.tgz", + "integrity": "sha512-uMgjozySS8adZZYePpaWs8cxB9/kdzmpX6SgJZ+wbz1K5eYk5QMYDVJaZKhxyIHUdnnJkfR7SVgStgH7LkGUyg==" + }, + "node_modules/anymatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.2.tgz", + "integrity": "sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==", + "dependencies": { + "normalize-path": "^3.0.0", + "picomatch": "^2.0.4" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/append-field": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/append-field/-/append-field-1.0.0.tgz", + "integrity": "sha1-HjRA6RXwsSA9I3SOeO3XubW0PlY=" + }, + "node_modules/application-config-path": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/application-config-path/-/application-config-path-0.1.0.tgz", + "integrity": "sha1-GTxfCoZUGkxm+6Hi3DhYM2LqXo8=" + }, + "node_modules/aproba": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/aproba/-/aproba-1.2.0.tgz", + "integrity": "sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw==" + }, + "node_modules/arch": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/arch/-/arch-2.2.0.tgz", + "integrity": "sha512-Of/R0wqp83cgHozfIYLbBMnej79U/SVGOOyuB3VVFv1NRM/PSFMK12x9KVtiYzJqmnU5WR2qp0Z5rHb7sWGnFQ==" + }, + "node_modules/are-we-there-yet": { + "version": "1.1.7", + "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-1.1.7.tgz", + "integrity": "sha512-nxwy40TuMiUGqMyRHgCSWZ9FM4VAoRP4xUYSTv5ImRog+h9yISPbVH7H8fASCIzYn9wlEv4zvFL7uKDMCFQm3g==", + "dependencies": { + "delegates": "^1.0.0", + "readable-stream": "^2.0.6" + } + }, + "node_modules/are-we-there-yet/node_modules/readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "node_modules/are-we-there-yet/node_modules/string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "dependencies": { + "safe-buffer": "~5.1.0" + } + }, + "node_modules/arg": { + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/arg/-/arg-4.1.3.tgz", + "integrity": "sha512-58S9QDqG0Xx27YwPSt9fJxivjYl432YCwfDMfZ+71RAqUrZef7LrKQZ3LHLOwCS4FLNBplP533Zx895SeOCHvA==" + }, + "node_modules/argparse": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", + "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", + "dependencies": { + "sprintf-js": "~1.0.2" + } + }, + "node_modules/aria-query": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/aria-query/-/aria-query-4.2.2.tgz", + "integrity": "sha512-o/HelwhuKpTj/frsOsbNLNgnNGVIFsVP/SW2BSF14gVl7kAfMOJ6/8wUAUvG1R1NHKrfG+2sHZTu0yauT1qBrA==", + "dependencies": { + "@babel/runtime": "^7.10.2", + "@babel/runtime-corejs3": "^7.10.2" + }, + "engines": { + "node": ">=6.0" + } + }, + "node_modules/arity-n": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/arity-n/-/arity-n-1.0.4.tgz", + "integrity": "sha512-fExL2kFDC1Q2DUOx3whE/9KoN66IzkY4b4zUHUBFM1ojEYjZZYDcUW3bek/ufGionX9giIKDC5redH2IlGqcQQ==" + }, + "node_modules/array-flatten": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-1.1.1.tgz", + "integrity": "sha1-ml9pkFGx5wczKPKgCJaLZOopVdI=" + }, + "node_modules/array-includes": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/array-includes/-/array-includes-3.1.5.tgz", + "integrity": "sha512-iSDYZMMyTPkiFasVqfuAQnWAYcvO/SeBSCGKePoEthjp4LEMTe4uLc7b025o4jAZpHhihh8xPo99TNWUWWkGDQ==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5", + "get-intrinsic": "^1.1.1", + "is-string": "^1.0.7" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/array-iterate": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/array-iterate/-/array-iterate-1.1.4.tgz", + "integrity": "sha512-sNRaPGh9nnmdC8Zf+pT3UqP8rnWj5Hf9wiFGsX3wUQ2yVSIhO2ShFwCoceIPpB41QF6i2OEmrHmCo36xronCVA==" + }, + "node_modules/array-union": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/array-union/-/array-union-2.1.0.tgz", + "integrity": "sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/array.prototype.flat": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/array.prototype.flat/-/array.prototype.flat-1.3.0.tgz", + "integrity": "sha512-12IUEkHsAhA4DY5s0FPgNXIdc8VRSqD9Zp78a5au9abH/SOBrsp082JOWFNTjkMozh8mqcdiKuaLGhPeYztxSw==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.2", + "es-shim-unscopables": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/array.prototype.flatmap": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/array.prototype.flatmap/-/array.prototype.flatmap-1.3.0.tgz", + "integrity": "sha512-PZC9/8TKAIxcWKdyeb77EzULHPrIX/tIZebLJUQOMR1OwYosT8yggdfWScfTBCDj5utONvOuPQQumYsU2ULbkg==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.2", + "es-shim-unscopables": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/arrify": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/arrify/-/arrify-2.0.1.tgz", + "integrity": "sha512-3duEwti880xqi4eAMN8AyR4a0ByT90zoYdLlevfrvU43vb0YZwZVfxOgxWrLXXXpyugL0hNZc9G6BiB5B3nUug==", + "engines": { + "node": ">=8" + } + }, + "node_modules/asap": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/asap/-/asap-2.0.6.tgz", + "integrity": "sha1-5QNHYR1+aQlDIIu9r+vLwvuGbUY=" + }, + "node_modules/asn1": { + "version": "0.2.6", + "resolved": "https://registry.npmmirror.com/asn1/-/asn1-0.2.6.tgz", + "integrity": "sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==", + "dependencies": { + "safer-buffer": "~2.1.0" + } + }, + "node_modules/assert-plus": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/assert-plus/-/assert-plus-1.0.0.tgz", + "integrity": "sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==", + "engines": { + "node": ">=0.8" + } + }, + "node_modules/ast-types-flow": { + "version": "0.0.7", + "resolved": "https://registry.npmjs.org/ast-types-flow/-/ast-types-flow-0.0.7.tgz", + "integrity": "sha1-9wtzXGvKGlycItmCw+Oef+ujva0=" + }, + "node_modules/astral-regex": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/astral-regex/-/astral-regex-2.0.0.tgz", + "integrity": "sha512-Z7tMw1ytTXt5jqMcOP+OQteU1VuNK9Y02uuJtKQ1Sv69jXQKKg5cibLwGJow8yzZP+eAc18EmLGPal0bp36rvQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/async": { + "version": "1.5.2", + "resolved": "https://registry.npmjs.org/async/-/async-1.5.2.tgz", + "integrity": "sha1-7GphrlZIDAw8skHJVhjiCJL5Zyo=" + }, + "node_modules/async-cache": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/async-cache/-/async-cache-1.1.0.tgz", + "integrity": "sha1-SppaidBl7F2OUlS9nulrp2xTK1o=", + "deprecated": "No longer maintained. Use [lru-cache](http://npm.im/lru-cache) version 7.6 or higher, and provide an asynchronous `fetchMethod` option.", + "dependencies": { + "lru-cache": "^4.0.0" + } + }, + "node_modules/async-cache/node_modules/lru-cache": { + "version": "4.1.5", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-4.1.5.tgz", + "integrity": "sha512-sWZlbEP2OsHNkXrMl5GYk/jKk70MBng6UU4YI/qGDYbgf6YbP4EvmqISbXCoJiRKs+1bSpFHVgQxvJ17F2li5g==", + "dependencies": { + "pseudomap": "^1.0.2", + "yallist": "^2.1.2" + } + }, + "node_modules/async-cache/node_modules/yallist": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-2.1.2.tgz", + "integrity": "sha1-HBH5IY8HYImkfdUS+TxmmaaoHVI=" + }, + "node_modules/async-foreach": { + "version": "0.1.3", + "resolved": "https://registry.npmmirror.com/async-foreach/-/async-foreach-0.1.3.tgz", + "integrity": "sha512-VUeSMD8nEGBWaZK4lizI1sf3yEC7pnAQ/mrI7pC2fBz2s/tq5jWWEngTwaf0Gruu/OoXRGLGg1XFqpYBiGTYJA==", + "engines": { + "node": "*" + } + }, + "node_modules/async-retry-ng": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/async-retry-ng/-/async-retry-ng-2.0.1.tgz", + "integrity": "sha512-iitlc2murdQ3/A5Re3CcplQBEf7vOmFrFQ6RFn3+/+zZUyIHYkZnnEziMSa6YIb2Bs2EJEPZWReTxjHqvQbDbw==" + }, + "node_modules/asynckit": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz", + "integrity": "sha1-x57Zf380y48robyXkLzDZkdLS3k=" + }, + "node_modules/at-least-node": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/at-least-node/-/at-least-node-1.0.0.tgz", + "integrity": "sha512-+q/t7Ekv1EDY2l6Gda6LLiX14rU9TV20Wa3ofeQmwPFZbOMo9DXrLbOjFaaclkXKWidIaopwAObQDqwWtGUjqg==", + "engines": { + "node": ">= 4.0.0" + } + }, + "node_modules/atob": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/atob/-/atob-2.1.2.tgz", + "integrity": "sha512-Wm6ukoaOGJi/73p/cl2GvLjTI5JM1k/O14isD73YML8StrH/7/lRFgmg8nICZgD3bZZvjwCGxtMOD3wWNAu8cg==", + "bin": { + "atob": "bin/atob.js" + }, + "engines": { + "node": ">= 4.5.0" + } + }, + "node_modules/auto-bind": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/auto-bind/-/auto-bind-4.0.0.tgz", + "integrity": "sha512-Hdw8qdNiqdJ8LqT0iK0sVzkFbzg6fhnQqqfWhBDxcHZvU75+B+ayzTy8x+k5Ix0Y92XOhOUlx74ps+bA6BeYMQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/autoprefixer": { + "version": "10.4.7", + "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-10.4.7.tgz", + "integrity": "sha512-ypHju4Y2Oav95SipEcCcI5J7CGPuvz8oat7sUtYj3ClK44bldfvtvcxK6IEK++7rqB7YchDGzweZIBG+SD0ZAA==", + "dependencies": { + "browserslist": "^4.20.3", + "caniuse-lite": "^1.0.30001335", + "fraction.js": "^4.2.0", + "normalize-range": "^0.1.2", + "picocolors": "^1.0.0", + "postcss-value-parser": "^4.2.0" + }, + "bin": { + "autoprefixer": "bin/autoprefixer" + }, + "engines": { + "node": "^10 || ^12 || >=14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/aws-sign2": { + "version": "0.7.0", + "resolved": "https://registry.npmmirror.com/aws-sign2/-/aws-sign2-0.7.0.tgz", + "integrity": "sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==", + "engines": { + "node": "*" + } + }, + "node_modules/aws4": { + "version": "1.11.0", + "resolved": "https://registry.npmmirror.com/aws4/-/aws4-1.11.0.tgz", + "integrity": "sha512-xh1Rl34h6Fi1DC2WWKfxUTVqRsNnr6LsKz2+hfwDxQJWmrx8+c7ylaqBMcHfl1U1r2dsifOvKX3LQuLNZ+XSvA==" + }, + "node_modules/axe-core": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/axe-core/-/axe-core-4.4.1.tgz", + "integrity": "sha512-gd1kmb21kwNuWr6BQz8fv6GNECPBnUasepcoLbekws23NVBLODdsClRZ+bQ8+9Uomf3Sm3+Vwn0oYG9NvwnJCw==", + "engines": { + "node": ">=4" + } + }, + "node_modules/axios": { + "version": "0.21.4", + "resolved": "https://registry.npmjs.org/axios/-/axios-0.21.4.tgz", + "integrity": "sha512-ut5vewkiu8jjGBdqpM44XxjuCjq9LAKeHVmoVfHVzy8eHgxxq8SbAVQNovDA8mVi05kP0Ea/n/UzcSHcTJQfNg==", + "dependencies": { + "follow-redirects": "^1.14.0" + } + }, + "node_modules/axobject-query": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/axobject-query/-/axobject-query-2.2.0.tgz", + "integrity": "sha512-Td525n+iPOOyUQIeBfcASuG6uJsDOITl7Mds5gFyerkWiX7qhUTdYUBlSgNMyVqtSJqwpt1kXGLdUt6SykLMRA==" + }, + "node_modules/babel-extract-comments": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/babel-extract-comments/-/babel-extract-comments-1.0.0.tgz", + "integrity": "sha512-qWWzi4TlddohA91bFwgt6zO/J0X+io7Qp184Fw0m2JYRSTZnJbFR8+07KmzudHCZgOiKRCrjhylwv9Xd8gfhVQ==", + "dependencies": { + "babylon": "^6.18.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/babel-jsx-utils": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/babel-jsx-utils/-/babel-jsx-utils-1.1.0.tgz", + "integrity": "sha512-Mh1j/rw4xM9T3YICkw22aBQ78FhsHdsmlb9NEk4uVAFBOg+Ez9ZgXXHugoBPCZui3XLomk/7/JBBH4daJqTkQQ==" + }, + "node_modules/babel-loader": { + "version": "8.2.5", + "resolved": "https://registry.npmjs.org/babel-loader/-/babel-loader-8.2.5.tgz", + "integrity": "sha512-OSiFfH89LrEMiWd4pLNqGz4CwJDtbs2ZVc+iGu2HrkRfPxId9F2anQj38IxWpmRfsUY0aBZYi1EFcd3mhtRMLQ==", + "dependencies": { + "find-cache-dir": "^3.3.1", + "loader-utils": "^2.0.0", + "make-dir": "^3.1.0", + "schema-utils": "^2.6.5" + }, + "engines": { + "node": ">= 8.9" + }, + "peerDependencies": { + "@babel/core": "^7.0.0", + "webpack": ">=2" + } + }, + "node_modules/babel-plugin-add-module-exports": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/babel-plugin-add-module-exports/-/babel-plugin-add-module-exports-1.0.4.tgz", + "integrity": "sha512-g+8yxHUZ60RcyaUpfNzy56OtWW+x9cyEe9j+CranqLiqbju2yf/Cy6ZtYK40EZxtrdHllzlVZgLmcOUCTlJ7Jg==" + }, + "node_modules/babel-plugin-dynamic-import-node": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/babel-plugin-dynamic-import-node/-/babel-plugin-dynamic-import-node-2.3.3.tgz", + "integrity": "sha512-jZVI+s9Zg3IqA/kdi0i6UDCybUI3aSBLnglhYbSSjKlV7yF1F/5LWv8MakQmvYpnbJDS6fcBL2KzHSxNCMtWSQ==", + "dependencies": { + "object.assign": "^4.1.0" + } + }, + "node_modules/babel-plugin-lodash": { + "version": "3.3.4", + "resolved": "https://registry.npmjs.org/babel-plugin-lodash/-/babel-plugin-lodash-3.3.4.tgz", + "integrity": "sha512-yDZLjK7TCkWl1gpBeBGmuaDIFhZKmkoL+Cu2MUUjv5VxUZx/z7tBGBCBcQs5RI1Bkz5LLmNdjx7paOyQtMovyg==", + "dependencies": { + "@babel/helper-module-imports": "^7.0.0-beta.49", + "@babel/types": "^7.0.0-beta.49", + "glob": "^7.1.1", + "lodash": "^4.17.10", + "require-package-name": "^2.0.1" + } + }, + "node_modules/babel-plugin-macros": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/babel-plugin-macros/-/babel-plugin-macros-3.1.0.tgz", + "integrity": "sha512-Cg7TFGpIr01vOQNODXOOaGz2NpCU5gl8x1qJFbb6hbZxR7XrcE2vtbAsTAbJ7/xwJtUuJEw8K8Zr/AE0LHlesg==", + "dependencies": { + "@babel/runtime": "^7.12.5", + "cosmiconfig": "^7.0.0", + "resolve": "^1.19.0" + }, + "engines": { + "node": ">=10", + "npm": ">=6" + } + }, + "node_modules/babel-plugin-polyfill-corejs2": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.3.1.tgz", + "integrity": "sha512-v7/T6EQcNfVLfcN2X8Lulb7DjprieyLWJK/zOWH5DUYcAgex9sP3h25Q+DLsX9TloXe3y1O8l2q2Jv9q8UVB9w==", + "dependencies": { + "@babel/compat-data": "^7.13.11", + "@babel/helper-define-polyfill-provider": "^0.3.1", + "semver": "^6.1.1" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/babel-plugin-polyfill-corejs2/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/babel-plugin-polyfill-corejs3": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.5.2.tgz", + "integrity": "sha512-G3uJih0XWiID451fpeFaYGVuxHEjzKTHtc9uGFEjR6hHrvNzeS/PX+LLLcetJcytsB5m4j+K3o/EpXJNb/5IEQ==", + "dependencies": { + "@babel/helper-define-polyfill-provider": "^0.3.1", + "core-js-compat": "^3.21.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/babel-plugin-polyfill-regenerator": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.3.1.tgz", + "integrity": "sha512-Y2B06tvgHYt1x0yz17jGkGeeMr5FeKUu+ASJ+N6nB5lQ8Dapfg42i0OVrf8PNGJ3zKL4A23snMi1IRwrqqND7A==", + "dependencies": { + "@babel/helper-define-polyfill-provider": "^0.3.1" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/babel-plugin-remove-graphql-queries": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/babel-plugin-remove-graphql-queries/-/babel-plugin-remove-graphql-queries-4.14.0.tgz", + "integrity": "sha512-rqCih6maArH0nbkndAP9UKKQCUWZy1NBxG+nSOoIZpvLkMqTweAuiTpMDJVHWDk9CycFlLfl09/Ayk/nciVKhA==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0", + "gatsby": "^4.0.0-next" + } + }, + "node_modules/babel-plugin-syntax-object-rest-spread": { + "version": "6.13.0", + "resolved": "https://registry.npmjs.org/babel-plugin-syntax-object-rest-spread/-/babel-plugin-syntax-object-rest-spread-6.13.0.tgz", + "integrity": "sha1-/WU28rzhODb/o6VFjEkDpZe7O/U=" + }, + "node_modules/babel-plugin-syntax-trailing-function-commas": { + "version": "7.0.0-beta.0", + "resolved": "https://registry.npmjs.org/babel-plugin-syntax-trailing-function-commas/-/babel-plugin-syntax-trailing-function-commas-7.0.0-beta.0.tgz", + "integrity": "sha512-Xj9XuRuz3nTSbaTXWv3itLOcxyF4oPD8douBBmj7U9BBC6nEBYfyOJYQMf/8PJAFotC62UY5dFfIGEPr7WswzQ==" + }, + "node_modules/babel-plugin-transform-object-rest-spread": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-plugin-transform-object-rest-spread/-/babel-plugin-transform-object-rest-spread-6.26.0.tgz", + "integrity": "sha1-DzZpLVD+9rfi1LOsFHgTepY7ewY=", + "dependencies": { + "babel-plugin-syntax-object-rest-spread": "^6.8.0", + "babel-runtime": "^6.26.0" + } + }, + "node_modules/babel-plugin-transform-react-remove-prop-types": { + "version": "0.4.24", + "resolved": "https://registry.npmjs.org/babel-plugin-transform-react-remove-prop-types/-/babel-plugin-transform-react-remove-prop-types-0.4.24.tgz", + "integrity": "sha512-eqj0hVcJUR57/Ug2zE1Yswsw4LhuqqHhD+8v120T1cl3kjg76QwtyBrdIk4WVwK+lAhBJVYCd/v+4nc4y+8JsA==" + }, + "node_modules/babel-preset-fbjs": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/babel-preset-fbjs/-/babel-preset-fbjs-3.4.0.tgz", + "integrity": "sha512-9ywCsCvo1ojrw0b+XYk7aFvTH6D9064t0RIL1rtMf3nsa02Xw41MS7sZw216Im35xj/UY0PDBQsa1brUDDF1Ow==", + "dependencies": { + "@babel/plugin-proposal-class-properties": "^7.0.0", + "@babel/plugin-proposal-object-rest-spread": "^7.0.0", + "@babel/plugin-syntax-class-properties": "^7.0.0", + "@babel/plugin-syntax-flow": "^7.0.0", + "@babel/plugin-syntax-jsx": "^7.0.0", + "@babel/plugin-syntax-object-rest-spread": "^7.0.0", + "@babel/plugin-transform-arrow-functions": "^7.0.0", + "@babel/plugin-transform-block-scoped-functions": "^7.0.0", + "@babel/plugin-transform-block-scoping": "^7.0.0", + "@babel/plugin-transform-classes": "^7.0.0", + "@babel/plugin-transform-computed-properties": "^7.0.0", + "@babel/plugin-transform-destructuring": "^7.0.0", + "@babel/plugin-transform-flow-strip-types": "^7.0.0", + "@babel/plugin-transform-for-of": "^7.0.0", + "@babel/plugin-transform-function-name": "^7.0.0", + "@babel/plugin-transform-literals": "^7.0.0", + "@babel/plugin-transform-member-expression-literals": "^7.0.0", + "@babel/plugin-transform-modules-commonjs": "^7.0.0", + "@babel/plugin-transform-object-super": "^7.0.0", + "@babel/plugin-transform-parameters": "^7.0.0", + "@babel/plugin-transform-property-literals": "^7.0.0", + "@babel/plugin-transform-react-display-name": "^7.0.0", + "@babel/plugin-transform-react-jsx": "^7.0.0", + "@babel/plugin-transform-shorthand-properties": "^7.0.0", + "@babel/plugin-transform-spread": "^7.0.0", + "@babel/plugin-transform-template-literals": "^7.0.0", + "babel-plugin-syntax-trailing-function-commas": "^7.0.0-beta.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/babel-preset-gatsby": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/babel-preset-gatsby/-/babel-preset-gatsby-2.14.0.tgz", + "integrity": "sha512-IqPgd15jJfJvqvX0i78JwLT48ctb7MdIEqHeKOuo4N8qWmyRIY8xX1IVhhSfDZ3eq62j0rVoqzT7ACUWctikmw==", + "dependencies": { + "@babel/plugin-proposal-class-properties": "^7.14.0", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.14.5", + "@babel/plugin-proposal-optional-chaining": "^7.14.5", + "@babel/plugin-syntax-dynamic-import": "^7.8.3", + "@babel/plugin-transform-classes": "^7.15.4", + "@babel/plugin-transform-runtime": "^7.15.0", + "@babel/plugin-transform-spread": "^7.14.6", + "@babel/preset-env": "^7.15.4", + "@babel/preset-react": "^7.14.0", + "@babel/runtime": "^7.15.4", + "babel-plugin-dynamic-import-node": "^2.3.3", + "babel-plugin-macros": "^3.1.0", + "babel-plugin-transform-react-remove-prop-types": "^0.4.24", + "gatsby-core-utils": "^3.14.0", + "gatsby-legacy-polyfills": "^2.14.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "@babel/core": "^7.11.6", + "core-js": "^3.0.0" + } + }, + "node_modules/babel-runtime": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-runtime/-/babel-runtime-6.26.0.tgz", + "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=", + "dependencies": { + "core-js": "^2.4.0", + "regenerator-runtime": "^0.11.0" + } + }, + "node_modules/babel-runtime/node_modules/core-js": { + "version": "2.6.12", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.6.12.tgz", + "integrity": "sha512-Kb2wC0fvsWfQrgk8HU5lW6U/Lcs8+9aaYcy4ZFc6DDlo4nZ7n70dEgE5rtR0oG6ufKDUnrwfWL1mXR5ljDatrQ==", + "deprecated": "core-js@<3.4 is no longer maintained and not recommended for usage due to the number of issues. Because of the V8 engine whims, feature detection in old core-js versions could cause a slowdown up to 100x even if nothing is polyfilled. Please, upgrade your dependencies to the actual version of core-js.", + "hasInstallScript": true + }, + "node_modules/babel-runtime/node_modules/regenerator-runtime": { + "version": "0.11.1", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.11.1.tgz", + "integrity": "sha512-MguG95oij0fC3QV3URf4V2SDYGJhJnJGqvIIgdECeODCT98wSWDAJ94SSuVpYQUoTcGUIL6L4yNB7j1DFFHSBg==" + }, + "node_modules/babylon": { + "version": "6.18.0", + "resolved": "https://registry.npmjs.org/babylon/-/babylon-6.18.0.tgz", + "integrity": "sha512-q/UEjfGJ2Cm3oKV71DJz9d25TPnq5rhBVL2Q4fA5wcC3jcrdn7+SssEybFIxwAvvP+YCsCYNKughoF33GxgycQ==", + "bin": { + "babylon": "bin/babylon.js" + } + }, + "node_modules/backo2": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/backo2/-/backo2-1.0.2.tgz", + "integrity": "sha1-MasayLEpNjRj41s+u2n038+6eUc=" + }, + "node_modules/bail": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/bail/-/bail-1.0.5.tgz", + "integrity": "sha512-xFbRxM1tahm08yHBP16MMjVUAvDaBMD38zsM9EMAUN61omwLmKlOpB/Zku5QkjZ8TZ4vn53pj+t518cH0S03RQ==" + }, + "node_modules/balanced-match": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + }, + "node_modules/base-x": { + "version": "3.0.9", + "resolved": "https://registry.npmjs.org/base-x/-/base-x-3.0.9.tgz", + "integrity": "sha512-H7JU6iBHTal1gp56aKoaa//YUxEaAOUiydvrV/pILqIHXTtqxSkATOnDA2u+jZ/61sD+L/412+7kzXRtWukhpQ==", + "dependencies": { + "safe-buffer": "^5.0.1" + } + }, + "node_modules/base64-arraybuffer": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/base64-arraybuffer/-/base64-arraybuffer-0.1.4.tgz", + "integrity": "sha1-mBjHngWbE1X5fgQooBfIOOkLqBI=", + "engines": { + "node": ">= 0.6.0" + } + }, + "node_modules/base64-js": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz", + "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==" + }, + "node_modules/base64id": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/base64id/-/base64id-2.0.0.tgz", + "integrity": "sha512-lGe34o6EHj9y3Kts9R4ZYs/Gr+6N7MCaMlIFA3F1R2O5/m7K06AxfSeO5530PEERE6/WyEg3lsuyw4GHlPZHog==", + "engines": { + "node": "^4.5.0 || >= 5.9" + } + }, + "node_modules/bcrypt-pbkdf": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz", + "integrity": "sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==", + "dependencies": { + "tweetnacl": "^0.14.3" + } + }, + "node_modules/better-opn": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/better-opn/-/better-opn-2.1.1.tgz", + "integrity": "sha512-kIPXZS5qwyKiX/HcRvDYfmBQUa8XP17I0mYZZ0y4UhpYOSvtsLHDYqmomS+Mj20aDvD3knEiQ0ecQy2nhio3yA==", + "dependencies": { + "open": "^7.0.3" + }, + "engines": { + "node": ">8.0.0" + } + }, + "node_modules/big.js": { + "version": "5.2.2", + "resolved": "https://registry.npmjs.org/big.js/-/big.js-5.2.2.tgz", + "integrity": "sha512-vyL2OymJxmarO8gxMr0mhChsO9QGwhynfuu4+MHTAW6czfq9humCB7rKpUjDd9YUiDPU4mzpyupFSvOClAwbmQ==", + "engines": { + "node": "*" + } + }, + "node_modules/binary-extensions": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", + "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/bl": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/bl/-/bl-4.1.0.tgz", + "integrity": "sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w==", + "dependencies": { + "buffer": "^5.5.0", + "inherits": "^2.0.4", + "readable-stream": "^3.4.0" + } + }, + "node_modules/bluebird": { + "version": "3.7.2", + "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz", + "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==" + }, + "node_modules/bmp-js": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/bmp-js/-/bmp-js-0.1.0.tgz", + "integrity": "sha1-4Fpj95amwf8l9Hcex62twUjAcjM=" + }, + "node_modules/body-parser": { + "version": "1.20.0", + "resolved": "https://registry.npmjs.org/body-parser/-/body-parser-1.20.0.tgz", + "integrity": "sha512-DfJ+q6EPcGKZD1QWUjSpqp+Q7bDQTsQIF4zfUAtZ6qk+H/3/QRhg9CEp39ss+/T2vw0+HaidC0ecJj/DRLIaKg==", + "dependencies": { + "bytes": "3.1.2", + "content-type": "~1.0.4", + "debug": "2.6.9", + "depd": "2.0.0", + "destroy": "1.2.0", + "http-errors": "2.0.0", + "iconv-lite": "0.4.24", + "on-finished": "2.4.1", + "qs": "6.10.3", + "raw-body": "2.5.1", + "type-is": "~1.6.18", + "unpipe": "1.0.0" + }, + "engines": { + "node": ">= 0.8", + "npm": "1.2.8000 || >= 1.4.16" + } + }, + "node_modules/body-parser/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/body-parser/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/boolbase": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/boolbase/-/boolbase-1.0.0.tgz", + "integrity": "sha1-aN/1++YMUes3cl6p4+0xDcwed24=" + }, + "node_modules/boxen": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/boxen/-/boxen-5.1.2.tgz", + "integrity": "sha512-9gYgQKXx+1nP8mP7CzFyaUARhg7D3n1dF/FnErWmu9l6JvGpNUN278h0aSb+QjoiKSWG+iZ3uHrcqk0qrY9RQQ==", + "dependencies": { + "ansi-align": "^3.0.0", + "camelcase": "^6.2.0", + "chalk": "^4.1.0", + "cli-boxes": "^2.2.1", + "string-width": "^4.2.2", + "type-fest": "^0.20.2", + "widest-line": "^3.1.0", + "wrap-ansi": "^7.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/boxen/node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/boxen/node_modules/camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/boxen/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/boxen/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/boxen/node_modules/wrap-ansi": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "dependencies": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dependencies": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "node_modules/braces": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", + "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "dependencies": { + "fill-range": "^7.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/browserslist": { + "version": "4.20.3", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.20.3.tgz", + "integrity": "sha512-NBhymBQl1zM0Y5dQT/O+xiLP9/rzOIQdKM/eMJBAq7yBgaB6krIYLGejrwVYnSHZdqjscB1SPuAjHwxjvN6Wdg==", + "dependencies": { + "caniuse-lite": "^1.0.30001332", + "electron-to-chromium": "^1.4.118", + "escalade": "^3.1.1", + "node-releases": "^2.0.3", + "picocolors": "^1.0.0" + }, + "bin": { + "browserslist": "cli.js" + }, + "engines": { + "node": "^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7" + } + }, + "node_modules/bser": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/bser/-/bser-2.1.1.tgz", + "integrity": "sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ==", + "dependencies": { + "node-int64": "^0.4.0" + } + }, + "node_modules/buffer": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/buffer/-/buffer-5.7.1.tgz", + "integrity": "sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ==", + "dependencies": { + "base64-js": "^1.3.1", + "ieee754": "^1.1.13" + } + }, + "node_modules/buffer-equal": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/buffer-equal/-/buffer-equal-0.0.1.tgz", + "integrity": "sha1-kbx0sR6kBbyRa8aqkI+q+ltKrEs=", + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/buffer-from": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/buffer-from/-/buffer-from-1.1.2.tgz", + "integrity": "sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==" + }, + "node_modules/busboy": { + "version": "0.2.14", + "resolved": "https://registry.npmjs.org/busboy/-/busboy-0.2.14.tgz", + "integrity": "sha1-bCpiLvz0fFe7vh4qnDetNseSVFM=", + "dependencies": { + "dicer": "0.2.5", + "readable-stream": "1.1.x" + }, + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/busboy/node_modules/isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8=" + }, + "node_modules/busboy/node_modules/readable-stream": { + "version": "1.1.14", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", + "integrity": "sha1-fPTFTvZI44EwhMY23SB54WbAgdk=", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "node_modules/busboy/node_modules/string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=" + }, + "node_modules/bytes": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/bytes/-/bytes-3.1.2.tgz", + "integrity": "sha512-/Nf7TyzTx6S3yRJObOAV7956r8cr2+Oj8AC5dt8wSP3BQAoeX58NoHyCU8P8zGkNXStjTSi6fzO6F0pBdcYbEg==", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/cache-manager": { + "version": "2.11.1", + "resolved": "https://registry.npmjs.org/cache-manager/-/cache-manager-2.11.1.tgz", + "integrity": "sha512-XhUuc9eYwkzpK89iNewFwtvcDYMUsvtwzHeyEOPJna/WsVsXcrzsA1ft2M0QqPNunEzLhNCYPo05tEfG+YuNow==", + "dependencies": { + "async": "1.5.2", + "lodash.clonedeep": "4.5.0", + "lru-cache": "4.0.0" + } + }, + "node_modules/cache-manager/node_modules/lru-cache": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-4.0.0.tgz", + "integrity": "sha1-tcvwFVbBaWb+vlTO7A+03JDfbCg=", + "dependencies": { + "pseudomap": "^1.0.1", + "yallist": "^2.0.0" + } + }, + "node_modules/cache-manager/node_modules/yallist": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-2.1.2.tgz", + "integrity": "sha1-HBH5IY8HYImkfdUS+TxmmaaoHVI=" + }, + "node_modules/cacheable-lookup": { + "version": "5.0.4", + "resolved": "https://registry.npmjs.org/cacheable-lookup/-/cacheable-lookup-5.0.4.tgz", + "integrity": "sha512-2/kNscPhpcxrOigMZzbiWF7dz8ilhb/nIHU3EyZiXWXpeq/au8qJ8VhdftMkty3n7Gj6HIGalQG8oiBNB3AJgA==", + "engines": { + "node": ">=10.6.0" + } + }, + "node_modules/cacheable-request": { + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-7.0.2.tgz", + "integrity": "sha512-pouW8/FmiPQbuGpkXQ9BAPv/Mo5xDGANgSNXzTzJ8DrKGuXOssM4wIQRjfanNRh3Yu5cfYPvcorqbhg2KIJtew==", + "dependencies": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^4.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^6.0.1", + "responselike": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/call-bind": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/call-bind/-/call-bind-1.0.2.tgz", + "integrity": "sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA==", + "dependencies": { + "function-bind": "^1.1.1", + "get-intrinsic": "^1.0.2" + } + }, + "node_modules/callsites": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/callsites/-/callsites-3.1.0.tgz", + "integrity": "sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/camel-case": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/camel-case/-/camel-case-4.1.2.tgz", + "integrity": "sha512-gxGWBrTT1JuMx6R+o5PTXMmUnhnVzLQ9SNutD4YqKtI6ap897t3tKECYla6gCWEkplXnlNybEkZg9GEGxKFCgw==", + "dependencies": { + "pascal-case": "^3.1.2", + "tslib": "^2.0.3" + } + }, + "node_modules/camelcase": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.3.1.tgz", + "integrity": "sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==", + "engines": { + "node": ">=6" + } + }, + "node_modules/camelcase-keys": { + "version": "6.2.2", + "resolved": "https://registry.npmmirror.com/camelcase-keys/-/camelcase-keys-6.2.2.tgz", + "integrity": "sha512-YrwaA0vEKazPBkn0ipTiMpSajYDSe+KjQfrjhcBMxJt/znbvlHd8Pw/Vamaz5EB4Wfhs3SUR3Z9mwRu/P3s3Yg==", + "dependencies": { + "camelcase": "^5.3.1", + "map-obj": "^4.0.0", + "quick-lru": "^4.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/camelcase-keys/node_modules/quick-lru": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/quick-lru/-/quick-lru-4.0.1.tgz", + "integrity": "sha512-ARhCpm70fzdcvNQfPoy49IaanKkTlRWF2JMzqhcJbhSFRZv7nPTvZJdcY7301IPmvW+/p0RgIWnQDLJxifsQ7g==", + "engines": { + "node": ">=8" + } + }, + "node_modules/caniuse-api": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/caniuse-api/-/caniuse-api-3.0.0.tgz", + "integrity": "sha512-bsTwuIg/BZZK/vreVTYYbSWoe2F+71P7K5QGEX+pT250DZbfU1MQ5prOKpPR+LL6uWKK3KMwMCAS74QB3Um1uw==", + "dependencies": { + "browserslist": "^4.0.0", + "caniuse-lite": "^1.0.0", + "lodash.memoize": "^4.1.2", + "lodash.uniq": "^4.5.0" + } + }, + "node_modules/caniuse-lite": { + "version": "1.0.30001339", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001339.tgz", + "integrity": "sha512-Es8PiVqCe+uXdms0Gu5xP5PF2bxLR7OBp3wUzUnuO7OHzhOfCyg3hdiGWVPVxhiuniOzng+hTc1u3fEQ0TlkSQ==" + }, + "node_modules/capital-case": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/capital-case/-/capital-case-1.0.4.tgz", + "integrity": "sha512-ds37W8CytHgwnhGGTi88pcPyR15qoNkOpYwmMMfnWqqWgESapLqvDx6huFjQ5vqWSn2Z06173XNA7LtMOeUh1A==", + "dependencies": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case-first": "^2.0.2" + } + }, + "node_modules/caseless": { + "version": "0.12.0", + "resolved": "https://registry.npmmirror.com/caseless/-/caseless-0.12.0.tgz", + "integrity": "sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==" + }, + "node_modules/ccount": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/ccount/-/ccount-1.1.0.tgz", + "integrity": "sha512-vlNK021QdI7PNeiUh/lKkC/mNHHfV0m/Ad5JoI0TYtlBnJAslM/JIkm/tGC88bkLIwO6OQ5uV6ztS6kVAtCDlg==" + }, + "node_modules/chalk": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", + "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "dependencies": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/chalk/node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/chalk/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/chalk/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/chalk/node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/chalk/node_modules/supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/change-case": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/change-case/-/change-case-4.1.2.tgz", + "integrity": "sha512-bSxY2ws9OtviILG1EiY5K7NNxkqg/JnRnFxLtKQ96JaviiIxi7djMrSd0ECT9AC+lttClmYwKw53BWpOMblo7A==", + "dependencies": { + "camel-case": "^4.1.2", + "capital-case": "^1.0.4", + "constant-case": "^3.0.4", + "dot-case": "^3.0.4", + "header-case": "^2.0.4", + "no-case": "^3.0.4", + "param-case": "^3.0.4", + "pascal-case": "^3.1.2", + "path-case": "^3.0.4", + "sentence-case": "^3.0.4", + "snake-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/change-case-all": { + "version": "1.0.14", + "resolved": "https://registry.npmjs.org/change-case-all/-/change-case-all-1.0.14.tgz", + "integrity": "sha512-CWVm2uT7dmSHdO/z1CXT/n47mWonyypzBbuCy5tN7uMg22BsfkhwT6oHmFCAk+gL1LOOxhdbB9SZz3J1KTY3gA==", + "dependencies": { + "change-case": "^4.1.2", + "is-lower-case": "^2.0.2", + "is-upper-case": "^2.0.2", + "lower-case": "^2.0.2", + "lower-case-first": "^2.0.2", + "sponge-case": "^1.0.1", + "swap-case": "^2.0.2", + "title-case": "^3.0.3", + "upper-case": "^2.0.2", + "upper-case-first": "^2.0.2" + } + }, + "node_modules/character-entities": { + "version": "1.2.4", + "resolved": "https://registry.npmmirror.com/character-entities/-/character-entities-1.2.4.tgz", + "integrity": "sha512-iBMyeEHxfVnIakwOuDXpVkc54HijNgCyQB2w0VfGQThle6NXn50zU6V/u+LDhxHcDUPojn6Kpga3PTAD8W1bQw==" + }, + "node_modules/character-entities-html4": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-entities-html4/-/character-entities-html4-1.1.4.tgz", + "integrity": "sha512-HRcDxZuZqMx3/a+qrzxdBKBPUpxWEq9xw2OPZ3a/174ihfrQKVsFhqtthBInFy1zZ9GgZyFXOatNujm8M+El3g==" + }, + "node_modules/character-entities-legacy": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-entities-legacy/-/character-entities-legacy-1.1.4.tgz", + "integrity": "sha512-3Xnr+7ZFS1uxeiUDvV02wQ+QDbc55o97tIV5zHScSPJpcLm/r0DFPcoY3tYRp+VZukxuMeKgXYmsXQHO05zQeA==" + }, + "node_modules/character-reference-invalid": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-reference-invalid/-/character-reference-invalid-1.1.4.tgz", + "integrity": "sha512-mKKUkUbhPpQlCOfIuZkvSEgktjPFIsZKRRbC6KWVEMvlzblj3i3asQv5ODsrwt0N3pHAEvjP8KTQPHkp0+6jOg==" + }, + "node_modules/chardet": { + "version": "0.7.0", + "resolved": "https://registry.npmjs.org/chardet/-/chardet-0.7.0.tgz", + "integrity": "sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA==" + }, + "node_modules/cheerio": { + "version": "1.0.0-rc.10", + "resolved": "https://registry.npmjs.org/cheerio/-/cheerio-1.0.0-rc.10.tgz", + "integrity": "sha512-g0J0q/O6mW8z5zxQ3A8E8J1hUgp4SMOvEoW/x84OwyHKe/Zccz83PVT4y5Crcr530FV6NgmKI1qvGTKVl9XXVw==", + "dependencies": { + "cheerio-select": "^1.5.0", + "dom-serializer": "^1.3.2", + "domhandler": "^4.2.0", + "htmlparser2": "^6.1.0", + "parse5": "^6.0.1", + "parse5-htmlparser2-tree-adapter": "^6.0.1", + "tslib": "^2.2.0" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/cheerio-select": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/cheerio-select/-/cheerio-select-1.6.0.tgz", + "integrity": "sha512-eq0GdBvxVFbqWgmCm7M3XGs1I8oLy/nExUnh6oLqmBditPO9AqQJrkslDpMun/hZ0yyTs8L0m85OHp4ho6Qm9g==", + "dependencies": { + "css-select": "^4.3.0", + "css-what": "^6.0.1", + "domelementtype": "^2.2.0", + "domhandler": "^4.3.1", + "domutils": "^2.8.0" + } + }, + "node_modules/chokidar": { + "version": "3.5.3", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", + "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "dependencies": { + "anymatch": "~3.1.2", + "braces": "~3.0.2", + "glob-parent": "~5.1.2", + "is-binary-path": "~2.1.0", + "is-glob": "~4.0.1", + "normalize-path": "~3.0.0", + "readdirp": "~3.6.0" + }, + "engines": { + "node": ">= 8.10.0" + }, + "optionalDependencies": { + "fsevents": "~2.3.2" + } + }, + "node_modules/chownr": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/chownr/-/chownr-1.1.4.tgz", + "integrity": "sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg==" + }, + "node_modules/chrome-trace-event": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz", + "integrity": "sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg==", + "engines": { + "node": ">=6.0" + } + }, + "node_modules/ci-info": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-2.0.0.tgz", + "integrity": "sha512-5tK7EtrZ0N+OLFMthtqOj4fI2Jeb88C4CAZPu25LDVUgXJ0A3Js4PMGqrn0JU1W0Mh1/Z8wZzYPxqUrXeBboCQ==" + }, + "node_modules/cli-boxes": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/cli-boxes/-/cli-boxes-2.2.1.tgz", + "integrity": "sha512-y4coMcylgSCdVinjiDBuR8PCC2bLjyGTwEmPb9NHR/QaNU6EUOXcTY/s6VjGMD6ENSEaeQYHCY0GNGS5jfMwPw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/cli-cursor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/cli-cursor/-/cli-cursor-3.1.0.tgz", + "integrity": "sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw==", + "dependencies": { + "restore-cursor": "^3.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/cli-width": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/cli-width/-/cli-width-3.0.0.tgz", + "integrity": "sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw==", + "engines": { + "node": ">= 10" + } + }, + "node_modules/clipboardy": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/clipboardy/-/clipboardy-2.3.0.tgz", + "integrity": "sha512-mKhiIL2DrQIsuXMgBgnfEHOZOryC7kY7YO//TN6c63wlEm3NG5tz+YgY5rVi29KCmq/QQjKYvM7a19+MDOTHOQ==", + "dependencies": { + "arch": "^2.1.1", + "execa": "^1.0.0", + "is-wsl": "^2.1.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/clipboardy/node_modules/execa": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/execa/-/execa-1.0.0.tgz", + "integrity": "sha512-adbxcyWV46qiHyvSp50TKt05tB4tK3HcmF7/nxfAdhnox83seTDbwnaqKO4sXRy7roHAIFqJP/Rw/AuEbX61LA==", + "dependencies": { + "cross-spawn": "^6.0.0", + "get-stream": "^4.0.0", + "is-stream": "^1.1.0", + "npm-run-path": "^2.0.0", + "p-finally": "^1.0.0", + "signal-exit": "^3.0.0", + "strip-eof": "^1.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/clipboardy/node_modules/get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/clipboardy/node_modules/is-stream": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-1.1.0.tgz", + "integrity": "sha1-EtSj3U5o4Lec6428hBc66A2RykQ=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/clipboardy/node_modules/npm-run-path": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-2.0.2.tgz", + "integrity": "sha1-NakjLfo11wZ7TLLd8jV7GHFTbF8=", + "dependencies": { + "path-key": "^2.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/cliui": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-6.0.0.tgz", + "integrity": "sha512-t6wbgtoCXvAzst7QgXxJYqPt0usEfbgQdftEPbLL/cvv6HPE5VgvqCuAIDR0NgU52ds6rFwqrgakNLrHEjCbrQ==", + "dependencies": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.0", + "wrap-ansi": "^6.2.0" + } + }, + "node_modules/clone": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/clone/-/clone-2.1.2.tgz", + "integrity": "sha1-G39Ln1kfHo+DZwQBYANFoCiHQ18=", + "engines": { + "node": ">=0.8" + } + }, + "node_modules/clone-deep": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/clone-deep/-/clone-deep-4.0.1.tgz", + "integrity": "sha512-neHB9xuzh/wk0dIHweyAXv2aPGZIVk3pLMe+/RNzINf17fe0OG96QroktYAUm7SM1PBnzTabaLboqqxDyMU+SQ==", + "dependencies": { + "is-plain-object": "^2.0.4", + "kind-of": "^6.0.2", + "shallow-clone": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/clone-response": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/clone-response/-/clone-response-1.0.2.tgz", + "integrity": "sha1-0dyXOSAxTfZ/vrlCI7TuNQI56Ws=", + "dependencies": { + "mimic-response": "^1.0.0" + } + }, + "node_modules/clsx": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/clsx/-/clsx-1.1.1.tgz", + "integrity": "sha512-6/bPho624p3S2pMyvP5kKBPXnI3ufHLObBFCfgx+LkeR5lg2XYy2hqZqUf45ypD8COn2bhgGJSUE+l5dhNBieA==", + "engines": { + "node": ">=6" + } + }, + "node_modules/coa": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/coa/-/coa-2.0.2.tgz", + "integrity": "sha512-q5/jG+YQnSy4nRTV4F7lPepBJZ8qBNJJDBuJdoejDyLXgmL7IEo+Le2JDZudFTFt7mrCqIRaSjws4ygRCTCAXA==", + "dependencies": { + "@types/q": "^1.5.1", + "chalk": "^2.4.1", + "q": "^1.1.2" + }, + "engines": { + "node": ">= 4.0" + } + }, + "node_modules/coa/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/code-point-at": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/code-point-at/-/code-point-at-1.1.0.tgz", + "integrity": "sha1-DQcLTQQ6W+ozovGkDi7bPZpMz3c=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/color": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/color/-/color-4.2.3.tgz", + "integrity": "sha512-1rXeuUUiGGrykh+CeBdu5Ie7OJwinCgQY0bc7GCRxy5xVHy+moaqkpL/jqQq0MtQOeYcrqEz4abc5f0KtU7W4A==", + "dependencies": { + "color-convert": "^2.0.1", + "color-string": "^1.9.0" + }, + "engines": { + "node": ">=12.5.0" + } + }, + "node_modules/color-convert": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.3.tgz", + "integrity": "sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==", + "dependencies": { + "color-name": "1.1.3" + } + }, + "node_modules/color-name": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.3.tgz", + "integrity": "sha1-p9BVi9icQveV3UIyj3QIMcpTvCU=" + }, + "node_modules/color-string": { + "version": "1.9.1", + "resolved": "https://registry.npmjs.org/color-string/-/color-string-1.9.1.tgz", + "integrity": "sha512-shrVawQFojnZv6xM40anx4CkoDP+fZsw/ZerEMsW/pyzsRbElpsL/DBVW7q3ExxwusdNXI3lXpuhEZkzs8p5Eg==", + "dependencies": { + "color-name": "^1.0.0", + "simple-swizzle": "^0.2.2" + } + }, + "node_modules/color/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/color/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/colord": { + "version": "2.9.2", + "resolved": "https://registry.npmjs.org/colord/-/colord-2.9.2.tgz", + "integrity": "sha512-Uqbg+J445nc1TKn4FoDPS6ZZqAvEDnwrH42yo8B40JSOgSLxMZ/gt3h4nmCtPLQeXhjJJkqBx7SCY35WnIixaQ==" + }, + "node_modules/colorette": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/colorette/-/colorette-1.4.0.tgz", + "integrity": "sha512-Y2oEozpomLn7Q3HFP7dpww7AtMJplbM9lGZP6RDfHqmbeRjiwRg4n6VM6j4KLmRke85uWEI7JqF17f3pqdRA0g==" + }, + "node_modules/combined-stream": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz", + "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==", + "dependencies": { + "delayed-stream": "~1.0.0" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/comma-separated-tokens": { + "version": "1.0.8", + "resolved": "https://registry.npmmirror.com/comma-separated-tokens/-/comma-separated-tokens-1.0.8.tgz", + "integrity": "sha512-GHuDRO12Sypu2cV70d1dkA2EUmXHgntrzbpvOB+Qy+49ypNfGgFQIC2fhhXbnyrJRynDCAARsT7Ou0M6hirpfw==" + }, + "node_modules/command-exists": { + "version": "1.2.9", + "resolved": "https://registry.npmjs.org/command-exists/-/command-exists-1.2.9.tgz", + "integrity": "sha512-LTQ/SGc+s0Xc0Fu5WaKnR0YiygZkm9eKFvyS+fRsU7/ZWFF8ykFM6Pc9aCVf1+xasOOZpO3BAVgVrKvsqKHV7w==" + }, + "node_modules/commander": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/commander/-/commander-7.2.0.tgz", + "integrity": "sha512-QrWXB+ZQSVPmIWIhtEO9H+gwHaMGYiF5ChvoJ+K9ZGHG/sVsa6yiesAD1GC/x46sET00Xlwo1u49RVVVzvcSkw==", + "engines": { + "node": ">= 10" + } + }, + "node_modules/common-tags": { + "version": "1.8.2", + "resolved": "https://registry.npmjs.org/common-tags/-/common-tags-1.8.2.tgz", + "integrity": "sha512-gk/Z852D2Wtb//0I+kRFNKKE9dIIVirjoqPoA1wJU+XePVXZfGeBpk45+A1rKO4Q43prqWBNY/MiIeRLbPWUaA==", + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/commondir": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/commondir/-/commondir-1.0.1.tgz", + "integrity": "sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs=" + }, + "node_modules/component-emitter": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.3.0.tgz", + "integrity": "sha512-Rd3se6QB+sO1TwqZjscQrurpEPIfO0/yYnSin6Q/rD3mOutHvUrCAhJub3r90uNb+SESBuE0QYoB90YdfatsRg==" + }, + "node_modules/compose-function": { + "version": "3.0.3", + "resolved": "https://registry.npmmirror.com/compose-function/-/compose-function-3.0.3.tgz", + "integrity": "sha512-xzhzTJ5eC+gmIzvZq+C3kCJHsp9os6tJkrigDRZclyGtOKINbZtE8n1Tzmeh32jW+BUDPbvZpibwvJHBLGMVwg==", + "dependencies": { + "arity-n": "^1.0.4" + } + }, + "node_modules/compress-brotli": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/compress-brotli/-/compress-brotli-1.3.8.tgz", + "integrity": "sha512-lVcQsjhxhIXsuupfy9fmZUFtAIdBmXA7EGY6GBdgZ++qkM9zG4YFT8iU7FoBxzryNDMOpD1HIFHUSX4D87oqhQ==", + "dependencies": { + "@types/json-buffer": "~3.0.0", + "json-buffer": "~3.0.1" + }, + "engines": { + "node": ">= 12" + } + }, + "node_modules/compressible": { + "version": "2.0.18", + "resolved": "https://registry.npmjs.org/compressible/-/compressible-2.0.18.tgz", + "integrity": "sha512-AF3r7P5dWxL8MxyITRMlORQNaOA2IkAFaTr4k7BUumjPtRpGDTZpl0Pb1XCO6JeDCBdp126Cgs9sMxqSjgYyRg==", + "dependencies": { + "mime-db": ">= 1.43.0 < 2" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/compression": { + "version": "1.7.4", + "resolved": "https://registry.npmjs.org/compression/-/compression-1.7.4.tgz", + "integrity": "sha512-jaSIDzP9pZVS4ZfQ+TzvtiWhdpFhE2RDHz8QJkpX9SIpLq88VueF5jJw6t+6CUQcAoA6t+x89MLrWAqpfDE8iQ==", + "dependencies": { + "accepts": "~1.3.5", + "bytes": "3.0.0", + "compressible": "~2.0.16", + "debug": "2.6.9", + "on-headers": "~1.0.2", + "safe-buffer": "5.1.2", + "vary": "~1.1.2" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/compression/node_modules/bytes": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/bytes/-/bytes-3.0.0.tgz", + "integrity": "sha1-0ygVQE1olpn4Wk6k+odV3ROpYEg=", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/compression/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/compression/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=" + }, + "node_modules/concat-stream": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/concat-stream/-/concat-stream-1.6.2.tgz", + "integrity": "sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw==", + "engines": [ + "node >= 0.8" + ], + "dependencies": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + } + }, + "node_modules/concat-stream/node_modules/readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "node_modules/concat-stream/node_modules/string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "dependencies": { + "safe-buffer": "~5.1.0" + } + }, + "node_modules/configstore": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/configstore/-/configstore-5.0.1.tgz", + "integrity": "sha512-aMKprgk5YhBNyH25hj8wGt2+D52Sw1DRRIzqBwLp2Ya9mFmY8KPvvtvmna8SxVR9JMZ4kzMD68N22vlaRpkeFA==", + "dependencies": { + "dot-prop": "^5.2.0", + "graceful-fs": "^4.1.2", + "make-dir": "^3.0.0", + "unique-string": "^2.0.0", + "write-file-atomic": "^3.0.0", + "xdg-basedir": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/confusing-browser-globals": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/confusing-browser-globals/-/confusing-browser-globals-1.0.11.tgz", + "integrity": "sha512-JsPKdmh8ZkmnHxDk55FZ1TqVLvEQTvoByJZRN9jzI0UjxK/QgAmsphz7PGtqgPieQZ/CQcHWXCR7ATDNhGe+YA==" + }, + "node_modules/console-control-strings": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz", + "integrity": "sha1-PXz0Rk22RG6mRL9LOVB/mFEAjo4=" + }, + "node_modules/constant-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/constant-case/-/constant-case-3.0.4.tgz", + "integrity": "sha512-I2hSBi7Vvs7BEuJDr5dDHfzb/Ruj3FyvFyh7KLilAjNQw3Be+xgqUBA2W6scVEcL0hL1dwPRtIqEPVUCKkSsyQ==", + "dependencies": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case": "^2.0.2" + } + }, + "node_modules/content-disposition": { + "version": "0.5.4", + "resolved": "https://registry.npmjs.org/content-disposition/-/content-disposition-0.5.4.tgz", + "integrity": "sha512-FveZTNuGw04cxlAiWbzi6zTAL/lhehaWbTtgluJh4/E95DqMwTmha3KZN1aAWA8cFIhHzMZUvLevkw5Rqk+tSQ==", + "dependencies": { + "safe-buffer": "5.2.1" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/content-disposition/node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + }, + "node_modules/content-type": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/content-type/-/content-type-1.0.4.tgz", + "integrity": "sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/convert-hrtime": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/convert-hrtime/-/convert-hrtime-3.0.0.tgz", + "integrity": "sha512-7V+KqSvMiHp8yWDuwfww06XleMWVVB9b9tURBx+G7UTADuo5hYPuowKloz4OzOqbPezxgo+fdQ1522WzPG4OeA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/convert-source-map": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-1.8.0.tgz", + "integrity": "sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA==", + "dependencies": { + "safe-buffer": "~5.1.1" + } + }, + "node_modules/cookie": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.4.2.tgz", + "integrity": "sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/cookie-signature": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/cookie-signature/-/cookie-signature-1.0.6.tgz", + "integrity": "sha1-4wOogrNCzD7oylE6eZmXNNqzriw=" + }, + "node_modules/core-js": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-3.22.5.tgz", + "integrity": "sha512-VP/xYuvJ0MJWRAobcmQ8F2H6Bsn+s7zqAAjFaHGBMc5AQm7zaelhD1LGduFn2EehEcQcU+br6t+fwbpQ5d1ZWA==", + "hasInstallScript": true + }, + "node_modules/core-js-compat": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js-compat/-/core-js-compat-3.22.5.tgz", + "integrity": "sha512-rEF75n3QtInrYICvJjrAgV03HwKiYvtKHdPtaba1KucG+cNZ4NJnH9isqt979e67KZlhpbCOTwnsvnIr+CVeOg==", + "dependencies": { + "browserslist": "^4.20.3", + "semver": "7.0.0" + } + }, + "node_modules/core-js-compat/node_modules/semver": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.0.0.tgz", + "integrity": "sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/core-js-pure": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js-pure/-/core-js-pure-3.22.5.tgz", + "integrity": "sha512-8xo9R00iYD7TcV7OrC98GwxiUEAabVWO3dix+uyWjnYrx9fyASLlIX+f/3p5dW5qByaP2bcZ8X/T47s55et/tA==", + "hasInstallScript": true + }, + "node_modules/core-util-is": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.3.tgz", + "integrity": "sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ==" + }, + "node_modules/cors": { + "version": "2.8.5", + "resolved": "https://registry.npmjs.org/cors/-/cors-2.8.5.tgz", + "integrity": "sha512-KIHbLJqu73RGr/hnbrO9uBeixNGuvSQjul/jdFvS/KFSIH1hWVd1ng7zOHx+YrEfInLG7q4n6GHQ9cDtxv/P6g==", + "dependencies": { + "object-assign": "^4", + "vary": "^1" + }, + "engines": { + "node": ">= 0.10" + } + }, + "node_modules/cosmiconfig": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.1.tgz", + "integrity": "sha512-a1YWNUV2HwGimB7dU2s1wUMurNKjpx60HxBB6xUM8Re+2s1g1IIfJvFR0/iCF+XHdE0GMTKTuLR32UQff4TEyQ==", + "dependencies": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.2.1", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.10.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/cosmiconfig-toml-loader": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig-toml-loader/-/cosmiconfig-toml-loader-1.0.0.tgz", + "integrity": "sha512-H/2gurFWVi7xXvCyvsWRLCMekl4tITJcX0QEsDMpzxtuxDyM59xLatYNg4s/k9AA/HdtCYfj2su8mgA0GSDLDA==", + "dependencies": { + "@iarna/toml": "^2.2.5" + } + }, + "node_modules/create-gatsby": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/create-gatsby/-/create-gatsby-2.14.0.tgz", + "integrity": "sha512-Q92Omw5zPTKRrv5XDcsIVzBqSIHwl3T1lpOjQhSrQd42LDKUFAuE8zf/kTWT0QXo9cacBC+diUWIRxkqIZVKzQ==", + "dependencies": { + "@babel/runtime": "^7.15.4" + }, + "bin": { + "create-gatsby": "cli.js" + } + }, + "node_modules/create-require": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/create-require/-/create-require-1.1.1.tgz", + "integrity": "sha512-dcKFX3jn0MpIaXjisoRvexIJVEKzaq7z2rZKxf+MSr9TkdmHmsU4m2lcLojrj/FHl8mk5VxMmYA+ftRkP/3oKQ==" + }, + "node_modules/cross-fetch": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/cross-fetch/-/cross-fetch-3.1.5.tgz", + "integrity": "sha512-lvb1SBsI0Z7GDwmuid+mU3kWVBwTVUbe7S0H52yaaAdQOXq2YktTCZdlAcNKFzE6QtRz0snpw9bNiPeOIkkQvw==", + "dependencies": { + "node-fetch": "2.6.7" + } + }, + "node_modules/cross-spawn": { + "version": "6.0.5", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-6.0.5.tgz", + "integrity": "sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ==", + "dependencies": { + "nice-try": "^1.0.4", + "path-key": "^2.0.1", + "semver": "^5.5.0", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + }, + "engines": { + "node": ">=4.8" + } + }, + "node_modules/cross-spawn/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/crypto-random-string": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/crypto-random-string/-/crypto-random-string-2.0.0.tgz", + "integrity": "sha512-v1plID3y9r/lPhviJ1wrXpLeyUIGAZ2SHNYTEapm7/8A9nLPoyvVp3RK/EPFqn5kEznyWgYZNsRtYYIWbuG8KA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/css": { + "version": "2.2.4", + "resolved": "https://registry.npmmirror.com/css/-/css-2.2.4.tgz", + "integrity": "sha512-oUnjmWpy0niI3x/mPL8dVEI1l7MnG3+HHyRPHf+YFSbK+svOhXpmSOcDURUh2aOCgl2grzrOPt1nHLuCVFULLw==", + "dependencies": { + "inherits": "^2.0.3", + "source-map": "^0.6.1", + "source-map-resolve": "^0.5.2", + "urix": "^0.1.0" + } + }, + "node_modules/css-declaration-sorter": { + "version": "6.2.2", + "resolved": "https://registry.npmjs.org/css-declaration-sorter/-/css-declaration-sorter-6.2.2.tgz", + "integrity": "sha512-Ufadglr88ZLsrvS11gjeu/40Lw74D9Am/Jpr3LlYm5Q4ZP5KdlUhG+6u2EjyXeZcxmZ2h1ebCKngDjolpeLHpg==", + "engines": { + "node": "^10 || ^12 || >=14" + }, + "peerDependencies": { + "postcss": "^8.0.9" + } + }, + "node_modules/css-loader": { + "version": "5.2.7", + "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-5.2.7.tgz", + "integrity": "sha512-Q7mOvpBNBG7YrVGMxRxcBJZFL75o+cH2abNASdibkj/fffYD8qWbInZrD0S9ccI6vZclF3DsHE7njGlLtaHbhg==", + "dependencies": { + "icss-utils": "^5.1.0", + "loader-utils": "^2.0.0", + "postcss": "^8.2.15", + "postcss-modules-extract-imports": "^3.0.0", + "postcss-modules-local-by-default": "^4.0.0", + "postcss-modules-scope": "^3.0.0", + "postcss-modules-values": "^4.0.0", + "postcss-value-parser": "^4.1.0", + "schema-utils": "^3.0.0", + "semver": "^7.3.5" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.27.0 || ^5.0.0" + } + }, + "node_modules/css-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/css-minimizer-webpack-plugin": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/css-minimizer-webpack-plugin/-/css-minimizer-webpack-plugin-2.0.0.tgz", + "integrity": "sha512-cG/uc94727tx5pBNtb1Sd7gvUPzwmcQi1lkpfqTpdkuNq75hJCw7bIVsCNijLm4dhDcr1atvuysl2rZqOG8Txw==", + "dependencies": { + "cssnano": "^5.0.0", + "jest-worker": "^26.3.0", + "p-limit": "^3.0.2", + "postcss": "^8.2.9", + "schema-utils": "^3.0.0", + "serialize-javascript": "^5.0.1", + "source-map": "^0.6.1" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^5.0.0" + }, + "peerDependenciesMeta": { + "clean-css": { + "optional": true + }, + "csso": { + "optional": true + } + } + }, + "node_modules/css-minimizer-webpack-plugin/node_modules/p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dependencies": { + "yocto-queue": "^0.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/css-minimizer-webpack-plugin/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/css-minimizer-webpack-plugin/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/css-select": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/css-select/-/css-select-4.3.0.tgz", + "integrity": "sha512-wPpOYtnsVontu2mODhA19JrqWxNsfdatRKd64kmpRbQgh1KtItko5sTnEpPdpSaJszTOhEMlF/RPz28qj4HqhQ==", + "dependencies": { + "boolbase": "^1.0.0", + "css-what": "^6.0.1", + "domhandler": "^4.3.1", + "domutils": "^2.8.0", + "nth-check": "^2.0.1" + } + }, + "node_modules/css-select-base-adapter": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/css-select-base-adapter/-/css-select-base-adapter-0.1.1.tgz", + "integrity": "sha512-jQVeeRG70QI08vSTwf1jHxp74JoZsr2XSgETae8/xC8ovSnL2WF87GTLO86Sbwdt2lK4Umg4HnnwMO4YF3Ce7w==" + }, + "node_modules/css-selector-parser": { + "version": "1.4.1", + "resolved": "https://registry.npmmirror.com/css-selector-parser/-/css-selector-parser-1.4.1.tgz", + "integrity": "sha512-HYPSb7y/Z7BNDCOrakL4raGO2zltZkbeXyAd6Tg9obzix6QhzxCotdBl6VT0Dv4vZfJGVz3WL/xaEI9Ly3ul0g==" + }, + "node_modules/css-tree": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.1.3.tgz", + "integrity": "sha512-tRpdppF7TRazZrjJ6v3stzv93qxRcSsFmW6cX0Zm2NVKpxE1WV1HblnghVv9TreireHkqI/VDEsfolRF1p6y7Q==", + "dependencies": { + "mdn-data": "2.0.14", + "source-map": "^0.6.1" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/css-tree/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/css-what": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/css-what/-/css-what-6.1.0.tgz", + "integrity": "sha512-HTUrgRJ7r4dsZKU6GjmpfRK1O76h97Z8MfS1G0FozR+oF2kG6Vfe8JE6zwrkbxigziPHinCJ+gCPjA9EaBDtRw==", + "engines": { + "node": ">= 6" + } + }, + "node_modules/css.escape": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/css.escape/-/css.escape-1.5.1.tgz", + "integrity": "sha1-QuJ9T6BK4y+TGktNQZH6nN3ul8s=" + }, + "node_modules/css/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/cssesc": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-3.0.0.tgz", + "integrity": "sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg==", + "bin": { + "cssesc": "bin/cssesc" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/cssfilter": { + "version": "0.0.10", + "resolved": "https://registry.npmjs.org/cssfilter/-/cssfilter-0.0.10.tgz", + "integrity": "sha1-xtJnJjKi5cg+AT5oZKQs6N79IK4=" + }, + "node_modules/cssnano": { + "version": "5.1.7", + "resolved": "https://registry.npmjs.org/cssnano/-/cssnano-5.1.7.tgz", + "integrity": "sha512-pVsUV6LcTXif7lvKKW9ZrmX+rGRzxkEdJuVJcp5ftUjWITgwam5LMZOgaTvUrWPkcORBey6he7JKb4XAJvrpKg==", + "dependencies": { + "cssnano-preset-default": "^5.2.7", + "lilconfig": "^2.0.3", + "yaml": "^1.10.2" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/cssnano-preset-default": { + "version": "5.2.7", + "resolved": "https://registry.npmjs.org/cssnano-preset-default/-/cssnano-preset-default-5.2.7.tgz", + "integrity": "sha512-JiKP38ymZQK+zVKevphPzNSGHSlTI+AOwlasoSRtSVMUU285O7/6uZyd5NbW92ZHp41m0sSHe6JoZosakj63uA==", + "dependencies": { + "css-declaration-sorter": "^6.2.2", + "cssnano-utils": "^3.1.0", + "postcss-calc": "^8.2.3", + "postcss-colormin": "^5.3.0", + "postcss-convert-values": "^5.1.0", + "postcss-discard-comments": "^5.1.1", + "postcss-discard-duplicates": "^5.1.0", + "postcss-discard-empty": "^5.1.1", + "postcss-discard-overridden": "^5.1.0", + "postcss-merge-longhand": "^5.1.4", + "postcss-merge-rules": "^5.1.1", + "postcss-minify-font-values": "^5.1.0", + "postcss-minify-gradients": "^5.1.1", + "postcss-minify-params": "^5.1.2", + "postcss-minify-selectors": "^5.2.0", + "postcss-normalize-charset": "^5.1.0", + "postcss-normalize-display-values": "^5.1.0", + "postcss-normalize-positions": "^5.1.0", + "postcss-normalize-repeat-style": "^5.1.0", + "postcss-normalize-string": "^5.1.0", + "postcss-normalize-timing-functions": "^5.1.0", + "postcss-normalize-unicode": "^5.1.0", + "postcss-normalize-url": "^5.1.0", + "postcss-normalize-whitespace": "^5.1.1", + "postcss-ordered-values": "^5.1.1", + "postcss-reduce-initial": "^5.1.0", + "postcss-reduce-transforms": "^5.1.0", + "postcss-svgo": "^5.1.0", + "postcss-unique-selectors": "^5.1.1" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/cssnano-utils": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/cssnano-utils/-/cssnano-utils-3.1.0.tgz", + "integrity": "sha512-JQNR19/YZhz4psLX/rQ9M83e3z2Wf/HdJbryzte4a3NSuafyp9w/I4U+hx5C2S9g41qlstH7DEWnZaaj83OuEA==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/csso": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/csso/-/csso-4.2.0.tgz", + "integrity": "sha512-wvlcdIbf6pwKEk7vHj8/Bkc0B4ylXZruLvOgs9doS5eOsOpuodOV2zJChSpkp+pRpYQLQMeF04nr3Z68Sta9jA==", + "dependencies": { + "css-tree": "^1.1.2" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/csstype": { + "version": "3.0.11", + "resolved": "https://registry.npmjs.org/csstype/-/csstype-3.0.11.tgz", + "integrity": "sha512-sa6P2wJ+CAbgyy4KFssIb/JNMLxFvKF1pCYCSXS8ZMuqZnMsrxqI2E5sPyoTpxoPU/gVZMzr2zjOfg8GIZOMsw==" + }, + "node_modules/d": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/d/-/d-1.0.1.tgz", + "integrity": "sha512-m62ShEObQ39CfralilEQRjH6oAMtNCV1xJyEx5LpRYUVN+EviphDgUc/F3hnYbADmkiNs67Y+3ylmlG7Lnu+FA==", + "dependencies": { + "es5-ext": "^0.10.50", + "type": "^1.0.1" + } + }, + "node_modules/damerau-levenshtein": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/damerau-levenshtein/-/damerau-levenshtein-1.0.8.tgz", + "integrity": "sha512-sdQSFB7+llfUcQHUQO3+B8ERRj0Oa4w9POWMI/puGtuf7gFywGmkaLCElnudfTiKZV+NvHqL0ifzdrI8Ro7ESA==" + }, + "node_modules/dashdash": { + "version": "1.14.1", + "resolved": "https://registry.npmmirror.com/dashdash/-/dashdash-1.14.1.tgz", + "integrity": "sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==", + "dependencies": { + "assert-plus": "^1.0.0" + }, + "engines": { + "node": ">=0.10" + } + }, + "node_modules/dataloader": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/dataloader/-/dataloader-2.0.0.tgz", + "integrity": "sha512-YzhyDAwA4TaQIhM5go+vCLmU0UikghC/t9DTQYZR2M/UvZ1MdOhPezSDZcjj9uqQJOMqjLcpWtyW2iNINdlatQ==" + }, + "node_modules/date-fns": { + "version": "2.28.0", + "resolved": "https://registry.npmjs.org/date-fns/-/date-fns-2.28.0.tgz", + "integrity": "sha512-8d35hViGYx/QH0icHYCeLmsLmMUheMmTyV9Fcm6gvNwdw31yXXH+O85sOBJ+OLnLQMKZowvpKb6FgMIQjcpvQw==", + "engines": { + "node": ">=0.11" + } + }, + "node_modules/debug": { + "version": "3.2.7", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", + "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", + "dependencies": { + "ms": "^2.1.1" + } + }, + "node_modules/decamelize": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-1.2.0.tgz", + "integrity": "sha1-9lNNFRSCabIDUue+4m9QH5oZEpA=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/decamelize-keys": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/decamelize-keys/-/decamelize-keys-1.1.0.tgz", + "integrity": "sha512-ocLWuYzRPoS9bfiSdDd3cxvrzovVMZnRDVEzAs+hWIVXGDbHxWMECij2OBuyB/An0FFW/nLuq6Kv1i/YC5Qfzg==", + "dependencies": { + "decamelize": "^1.1.0", + "map-obj": "^1.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/decamelize-keys/node_modules/map-obj": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/map-obj/-/map-obj-1.0.1.tgz", + "integrity": "sha512-7N/q3lyZ+LVCp7PzuxrJr4KMbBE2hW7BT7YNia330OFxIf4d3r5zVpicP2650l7CPN6RM9zOJRl3NGpqSiw3Eg==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/decode-uri-component": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/decode-uri-component/-/decode-uri-component-0.2.0.tgz", + "integrity": "sha1-6zkTMzRYd1y4TNGh+uBiEGu4dUU=", + "engines": { + "node": ">=0.10" + } + }, + "node_modules/decompress-response": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-6.0.0.tgz", + "integrity": "sha512-aW35yZM6Bb/4oJlZncMH2LCoZtJXTRxES17vE3hoRiowU2kWHaJKFkSBDnDR+cm9J+9QhXmREyIfv0pji9ejCQ==", + "dependencies": { + "mimic-response": "^3.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/decompress-response/node_modules/mimic-response": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/mimic-response/-/mimic-response-3.1.0.tgz", + "integrity": "sha512-z0yWI+4FDrrweS8Zmt4Ej5HdJmky15+L2e6Wgn3+iK5fWzb6T3fhNFq2+MeTRb064c6Wr4N/wv0DzQTjNzHNGQ==", + "engines": { + "node": ">=10" + } + }, + "node_modules/deep-extend": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/deep-extend/-/deep-extend-0.6.0.tgz", + "integrity": "sha512-LOHxIOaPYdHlJRtCQfDIVZtfw/ufM8+rVj649RIHzcm/vGwQRXFt6OPqIFWsm2XEMrNIEtWR64sY1LEKD2vAOA==", + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/deep-is": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", + "integrity": "sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==" + }, + "node_modules/deepmerge": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/deepmerge/-/deepmerge-4.2.2.tgz", + "integrity": "sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/defer-to-connect": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-2.0.1.tgz", + "integrity": "sha512-4tvttepXG1VaYGrRibk5EwJd1t4udunSOVMdLSAL6mId1ix438oPwPZMALY41FCijukO1L0twNcGsdzS7dHgDg==", + "engines": { + "node": ">=10" + } + }, + "node_modules/define-lazy-prop": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz", + "integrity": "sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og==", + "engines": { + "node": ">=8" + } + }, + "node_modules/define-properties": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.1.4.tgz", + "integrity": "sha512-uckOqKcfaVvtBdsVkdPv3XjveQJsNQqmhXgRi8uhvWWuPYZCNlzT8qAyblUgNoXdHdjMTzAqeGjAoli8f+bzPA==", + "dependencies": { + "has-property-descriptors": "^1.0.0", + "object-keys": "^1.1.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/delayed-stream": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", + "integrity": "sha1-3zrhmayt+31ECqrgsp4icrJOxhk=", + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/delegates": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", + "integrity": "sha1-hMbhWbgZBP3KWaDvRM2HDTElD5o=" + }, + "node_modules/depd": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/depd/-/depd-2.0.0.tgz", + "integrity": "sha512-g7nH6P6dyDioJogAAGprGpCtVImJhpPk/roCzdb3fIh61/s/nPsfR6onyMwkCAR/OlC3yBC0lESvUoQEAssIrw==", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/dependency-graph": { + "version": "0.11.0", + "resolved": "https://registry.npmjs.org/dependency-graph/-/dependency-graph-0.11.0.tgz", + "integrity": "sha512-JeMq7fEshyepOWDfcfHK06N3MhyPhz++vtqWhMT5O9A3K42rdsEDpfdVqjaqaAhsw6a+ZqeDvQVtD0hFHQWrzg==", + "engines": { + "node": ">= 0.6.0" + } + }, + "node_modules/destroy": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/destroy/-/destroy-1.2.0.tgz", + "integrity": "sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg==", + "engines": { + "node": ">= 0.8", + "npm": "1.2.8000 || >= 1.4.16" + } + }, + "node_modules/detect-libc": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-1.0.3.tgz", + "integrity": "sha1-+hN8S9aY7fVc1c0CrFWfkaTEups=", + "bin": { + "detect-libc": "bin/detect-libc.js" + }, + "engines": { + "node": ">=0.10" + } + }, + "node_modules/detect-port": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/detect-port/-/detect-port-1.3.0.tgz", + "integrity": "sha512-E+B1gzkl2gqxt1IhUzwjrxBKRqx1UzC3WLONHinn8S3T6lwV/agVCyitiFOsGJ/eYuEUBvD71MZHy3Pv1G9doQ==", + "dependencies": { + "address": "^1.0.1", + "debug": "^2.6.0" + }, + "bin": { + "detect": "bin/detect-port", + "detect-port": "bin/detect-port" + }, + "engines": { + "node": ">= 4.2.1" + } + }, + "node_modules/detect-port/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/detect-port/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/devcert": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/devcert/-/devcert-1.2.1.tgz", + "integrity": "sha512-R7DqtMtsNmFVY75kzRHXON3hXoJili2xxlEcZgHi0VHSx8aJECfm7ZqAquXzTeAM/I9f8G2pHc/zq5k6iXHQzA==", + "dependencies": { + "@types/configstore": "^2.1.1", + "@types/debug": "^0.0.30", + "@types/get-port": "^3.2.0", + "@types/glob": "^5.0.34", + "@types/lodash": "^4.14.92", + "@types/mkdirp": "^0.5.2", + "@types/node": "^8.5.7", + "@types/rimraf": "^2.0.2", + "@types/tmp": "^0.0.33", + "application-config-path": "^0.1.0", + "command-exists": "^1.2.4", + "debug": "^3.1.0", + "eol": "^0.9.1", + "get-port": "^3.2.0", + "glob": "^7.1.2", + "is-valid-domain": "^0.1.6", + "lodash": "^4.17.4", + "mkdirp": "^0.5.1", + "password-prompt": "^1.0.4", + "rimraf": "^2.6.2", + "sudo-prompt": "^8.2.0", + "tmp": "^0.0.33", + "tslib": "^1.10.0" + } + }, + "node_modules/devcert/node_modules/@types/node": { + "version": "8.10.66", + "resolved": "https://registry.npmjs.org/@types/node/-/node-8.10.66.tgz", + "integrity": "sha512-tktOkFUA4kXx2hhhrB8bIFb5TbwzS4uOhKEmwiD+NoiL0qtP2OQ9mFldbgD4dV1djrlBYP6eBuQZiWjuHUpqFw==" + }, + "node_modules/devcert/node_modules/rimraf": { + "version": "2.7.1", + "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.7.1.tgz", + "integrity": "sha512-uWjbaKIK3T1OSVptzX7Nl6PvQ3qAGtKEtVRjRuazjfL3Bx5eI409VZSqgND+4UNnmzLVdPj9FqFJNPqBZFve4w==", + "dependencies": { + "glob": "^7.1.3" + }, + "bin": { + "rimraf": "bin.js" + } + }, + "node_modules/devcert/node_modules/tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==", + "dependencies": { + "os-tmpdir": "~1.0.2" + }, + "engines": { + "node": ">=0.6.0" + } + }, + "node_modules/devcert/node_modules/tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + }, + "node_modules/dicer": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/dicer/-/dicer-0.2.5.tgz", + "integrity": "sha1-WZbAhrszIYyBLAkL3cCc0S+stw8=", + "dependencies": { + "readable-stream": "1.1.x", + "streamsearch": "0.1.2" + }, + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/dicer/node_modules/isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8=" + }, + "node_modules/dicer/node_modules/readable-stream": { + "version": "1.1.14", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", + "integrity": "sha1-fPTFTvZI44EwhMY23SB54WbAgdk=", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "node_modules/dicer/node_modules/string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=" + }, + "node_modules/diff": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/diff/-/diff-4.0.2.tgz", + "integrity": "sha512-58lmxKSA4BNyLz+HHMUzlOEpg09FV+ev6ZMe3vJihgdxzgcwZ8VoEEPmALCZG9LmqfVoNMMKpttIYTVG6uDY7A==", + "engines": { + "node": ">=0.3.1" + } + }, + "node_modules/dir-glob": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-3.0.1.tgz", + "integrity": "sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==", + "dependencies": { + "path-type": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/doctrine": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", + "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/dom-converter": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/dom-converter/-/dom-converter-0.2.0.tgz", + "integrity": "sha512-gd3ypIPfOMr9h5jIKq8E3sHOTCjeirnl0WK5ZdS1AW0Odt0b1PaWaHdJ4Qk4klv+YB9aJBS7mESXjFoDQPu6DA==", + "dependencies": { + "utila": "~0.4" + } + }, + "node_modules/dom-serializer": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-1.4.1.tgz", + "integrity": "sha512-VHwB3KfrcOOkelEG2ZOfxqLZdfkil8PtJi4P8N2MMXucZq2yLp75ClViUlOVwyoHEDjYU433Aq+5zWP61+RGag==", + "dependencies": { + "domelementtype": "^2.0.1", + "domhandler": "^4.2.0", + "entities": "^2.0.0" + } + }, + "node_modules/dom-walk": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/dom-walk/-/dom-walk-0.1.2.tgz", + "integrity": "sha512-6QvTW9mrGeIegrFXdtQi9pk7O/nSK6lSdXW2eqUspN5LWD7UTji2Fqw5V2YLjBpHEoU9Xl/eUWNpDeZvoyOv2w==" + }, + "node_modules/domelementtype": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-2.3.0.tgz", + "integrity": "sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw==" + }, + "node_modules/domhandler": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-4.3.1.tgz", + "integrity": "sha512-GrwoxYN+uWlzO8uhUXRl0P+kHE4GtVPfYzVLcUxPL7KNdHKj66vvlhiweIHqYYXWlw+T8iLMp42Lm67ghw4WMQ==", + "dependencies": { + "domelementtype": "^2.2.0" + }, + "engines": { + "node": ">= 4" + } + }, + "node_modules/domutils": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/domutils/-/domutils-2.8.0.tgz", + "integrity": "sha512-w96Cjofp72M5IIhpjgobBimYEfoPjx1Vx0BSX9P30WBdZW2WIKU0T1Bd0kz2eNZ9ikjKgHbEyKx8BB6H1L3h3A==", + "dependencies": { + "dom-serializer": "^1.0.1", + "domelementtype": "^2.2.0", + "domhandler": "^4.2.0" + } + }, + "node_modules/dot-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/dot-case/-/dot-case-3.0.4.tgz", + "integrity": "sha512-Kv5nKlh6yRrdrGvxeJ2e5y2eRUpkUosIW4A2AS38zwSz27zu7ufDwQPi5Jhs3XAlGNetl3bmnGhQsMtkKJnj3w==", + "dependencies": { + "no-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/dot-prop": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/dot-prop/-/dot-prop-5.3.0.tgz", + "integrity": "sha512-QM8q3zDe58hqUqjraQOmzZ1LIH9SWQJTlEKCH4kJ2oQvLZk7RbQXvtDM2XEq3fwkV9CCvvH4LA0AV+ogFsBM2Q==", + "dependencies": { + "is-obj": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/dotenv": { + "version": "8.6.0", + "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-8.6.0.tgz", + "integrity": "sha512-IrPdXQsk2BbzvCBGBOTmmSH5SodmqZNt4ERAZDmW4CT+tL8VtvinqywuANaFu4bOMWki16nqf0e4oC0QIaDr/g==", + "engines": { + "node": ">=10" + } + }, + "node_modules/dotenv-expand": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/dotenv-expand/-/dotenv-expand-5.1.0.tgz", + "integrity": "sha512-YXQl1DSa4/PQyRfgrv6aoNjhasp/p4qs9FjJ4q4cQk+8m4r6k4ZSiEyytKG8f8W9gi8WsQtIObNmKd+tMzNTmA==" + }, + "node_modules/duplexer": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.2.tgz", + "integrity": "sha512-jtD6YG370ZCIi/9GTaJKQxWTZD045+4R4hTk/x1UyoqadyJ9x9CgSi1RlVDQF8U2sxLLSnFkCaMihqljHIWgMg==" + }, + "node_modules/duplexer2": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/duplexer2/-/duplexer2-0.1.4.tgz", + "integrity": "sha512-asLFVfWWtJ90ZyOUHMqk7/S2w2guQKxUI2itj3d92ADHhxUSbCMGi1f1cBcJ7xM1To+pE/Khbwo1yuNbMEPKeA==", + "dependencies": { + "readable-stream": "^2.0.2" + } + }, + "node_modules/duplexer2/node_modules/readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "node_modules/duplexer2/node_modules/string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "dependencies": { + "safe-buffer": "~5.1.0" + } + }, + "node_modules/duplexer3": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/duplexer3/-/duplexer3-0.1.4.tgz", + "integrity": "sha1-7gHdHKwO08vH/b6jfcCo8c4ALOI=" + }, + "node_modules/ecc-jsbn": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz", + "integrity": "sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==", + "dependencies": { + "jsbn": "~0.1.0", + "safer-buffer": "^2.1.0" + } + }, + "node_modules/ee-first": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/ee-first/-/ee-first-1.1.1.tgz", + "integrity": "sha1-WQxhFWsK4vTwJVcyoViyZrxWsh0=" + }, + "node_modules/electron-to-chromium": { + "version": "1.4.137", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.137.tgz", + "integrity": "sha512-0Rcpald12O11BUogJagX3HsCN3FE83DSqWjgXoHo5a72KUKMSfI39XBgJpgNNxS9fuGzytaFjE06kZkiVFy2qA==" + }, + "node_modules/emoji-regex": { + "version": "8.0.0", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" + }, + "node_modules/emojis-list": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/emojis-list/-/emojis-list-3.0.0.tgz", + "integrity": "sha512-/kyM18EfinwXZbno9FyUGeFh87KC8HRQBQGildHZbEuRyWFOmv1U10o9BBp8XVZDVNNuQKyIGIu5ZYAAXJ0V2Q==", + "engines": { + "node": ">= 4" + } + }, + "node_modules/encodeurl": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-1.0.2.tgz", + "integrity": "sha1-rT/0yG7C0CkyL1oCw6mmBslbP1k=", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/end-of-stream": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.4.tgz", + "integrity": "sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==", + "dependencies": { + "once": "^1.4.0" + } + }, + "node_modules/engine.io": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/engine.io/-/engine.io-4.1.2.tgz", + "integrity": "sha512-t5z6zjXuVLhXDMiFJPYsPOWEER8B0tIsD3ETgw19S1yg9zryvUfY3Vhtk3Gf4sihw/bQGIqQ//gjvVlu+Ca0bQ==", + "dependencies": { + "accepts": "~1.3.4", + "base64id": "2.0.0", + "cookie": "~0.4.1", + "cors": "~2.8.5", + "debug": "~4.3.1", + "engine.io-parser": "~4.0.0", + "ws": "~7.4.2" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/engine.io-client": { + "version": "4.1.4", + "resolved": "https://registry.npmjs.org/engine.io-client/-/engine.io-client-4.1.4.tgz", + "integrity": "sha512-843fqAdKeUMFqKi1sSjnR11tJ4wi8sIefu6+JC1OzkkJBmjtc/gM/rZ53tJfu5Iae/3gApm5veoS+v+gtT0+Fg==", + "dependencies": { + "base64-arraybuffer": "0.1.4", + "component-emitter": "~1.3.0", + "debug": "~4.3.1", + "engine.io-parser": "~4.0.1", + "has-cors": "1.1.0", + "parseqs": "0.0.6", + "parseuri": "0.0.6", + "ws": "~7.4.2", + "xmlhttprequest-ssl": "~1.6.2", + "yeast": "0.1.2" + } + }, + "node_modules/engine.io-client/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/engine.io-parser": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/engine.io-parser/-/engine.io-parser-4.0.3.tgz", + "integrity": "sha512-xEAAY0msNnESNPc00e19y5heTPX4y/TJ36gr8t1voOaNmTojP9b3oK3BbJLFufW2XFPQaaijpFewm2g2Um3uqA==", + "dependencies": { + "base64-arraybuffer": "0.1.4" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/engine.io/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/enhanced-resolve": { + "version": "5.9.3", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.9.3.tgz", + "integrity": "sha512-Bq9VSor+kjvW3f9/MiiR4eE3XYgOl7/rS8lnSxbRbF3kS0B2r+Y9w5krBWxZgDxASVZbdYrn5wT4j/Wb0J9qow==", + "dependencies": { + "graceful-fs": "^4.2.4", + "tapable": "^2.2.0" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/enquirer": { + "version": "2.3.6", + "resolved": "https://registry.npmjs.org/enquirer/-/enquirer-2.3.6.tgz", + "integrity": "sha512-yjNnPr315/FjS4zIsUxYguYUPP2e1NK4d7E7ZOLiyYCcbFBiTMyID+2wvm2w6+pZ/odMA7cRkjhsPbltwBOrLg==", + "dependencies": { + "ansi-colors": "^4.1.1" + }, + "engines": { + "node": ">=8.6" + } + }, + "node_modules/entities": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/entities/-/entities-2.2.0.tgz", + "integrity": "sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A==" + }, + "node_modules/env-paths": { + "version": "2.2.1", + "resolved": "https://registry.npmmirror.com/env-paths/-/env-paths-2.2.1.tgz", + "integrity": "sha512-+h1lkLKhZMTYjog1VEpJNG7NZJWcuc2DDk/qsqSTRRCOXiLjeQ1d1/udrUGhqMxUgAlwKNZ0cf2uqan5GLuS2A==", + "engines": { + "node": ">=6" + } + }, + "node_modules/envinfo": { + "version": "7.8.1", + "resolved": "https://registry.npmjs.org/envinfo/-/envinfo-7.8.1.tgz", + "integrity": "sha512-/o+BXHmB7ocbHEAs6F2EnG0ogybVVUdkRunTT2glZU9XAaGmhqskrvKwqXuDfNjEO0LZKWdejEEpnq8aM0tOaw==", + "bin": { + "envinfo": "dist/cli.js" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eol": { + "version": "0.9.1", + "resolved": "https://registry.npmjs.org/eol/-/eol-0.9.1.tgz", + "integrity": "sha512-Ds/TEoZjwggRoz/Q2O7SE3i4Jm66mqTDfmdHdq/7DKVk3bro9Q8h6WdXKdPqFLMoqxrDK5SVRzHVPOS6uuGtrg==" + }, + "node_modules/error-ex": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/error-ex/-/error-ex-1.3.2.tgz", + "integrity": "sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g==", + "dependencies": { + "is-arrayish": "^0.2.1" + } + }, + "node_modules/error-stack-parser": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/error-stack-parser/-/error-stack-parser-2.0.7.tgz", + "integrity": "sha512-chLOW0ZGRf4s8raLrDxa5sdkvPec5YdvwbFnqJme4rk0rFajP8mPtrDL1+I+CwrQDCjswDA5sREX7jYQDQs9vA==", + "dependencies": { + "stackframe": "^1.1.1" + } + }, + "node_modules/es-abstract": { + "version": "1.20.0", + "resolved": "https://registry.npmjs.org/es-abstract/-/es-abstract-1.20.0.tgz", + "integrity": "sha512-URbD8tgRthKD3YcC39vbvSDrX23upXnPcnGAjQfgxXF5ID75YcENawc9ZX/9iTP9ptUyfCLIxTTuMYoRfiOVKA==", + "dependencies": { + "call-bind": "^1.0.2", + "es-to-primitive": "^1.2.1", + "function-bind": "^1.1.1", + "function.prototype.name": "^1.1.5", + "get-intrinsic": "^1.1.1", + "get-symbol-description": "^1.0.0", + "has": "^1.0.3", + "has-property-descriptors": "^1.0.0", + "has-symbols": "^1.0.3", + "internal-slot": "^1.0.3", + "is-callable": "^1.2.4", + "is-negative-zero": "^2.0.2", + "is-regex": "^1.1.4", + "is-shared-array-buffer": "^1.0.2", + "is-string": "^1.0.7", + "is-weakref": "^1.0.2", + "object-inspect": "^1.12.0", + "object-keys": "^1.1.1", + "object.assign": "^4.1.2", + "regexp.prototype.flags": "^1.4.1", + "string.prototype.trimend": "^1.0.5", + "string.prototype.trimstart": "^1.0.5", + "unbox-primitive": "^1.0.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/es-module-lexer": { + "version": "0.9.3", + "resolved": "https://registry.npmjs.org/es-module-lexer/-/es-module-lexer-0.9.3.tgz", + "integrity": "sha512-1HQ2M2sPtxwnvOvT1ZClHyQDiggdNjURWpY2we6aMKCQiUVxTmVs2UYPLIrD84sS+kMdUwfBSylbJPwNnBrnHQ==" + }, + "node_modules/es-shim-unscopables": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/es-shim-unscopables/-/es-shim-unscopables-1.0.0.tgz", + "integrity": "sha512-Jm6GPcCdC30eMLbZ2x8z2WuRwAws3zTBBKuusffYVUrNj/GVSUAZ+xKMaUpfNDR5IbyNA5LJbaecoUVbmUcB1w==", + "dependencies": { + "has": "^1.0.3" + } + }, + "node_modules/es-to-primitive": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/es-to-primitive/-/es-to-primitive-1.2.1.tgz", + "integrity": "sha512-QCOllgZJtaUo9miYBcLChTUaHNjJF3PYs1VidD7AwiEj1kYxKeQTctLAezAOH5ZKRH0g2IgPn6KwB4IT8iRpvA==", + "dependencies": { + "is-callable": "^1.1.4", + "is-date-object": "^1.0.1", + "is-symbol": "^1.0.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/es5-ext": { + "version": "0.10.61", + "resolved": "https://registry.npmjs.org/es5-ext/-/es5-ext-0.10.61.tgz", + "integrity": "sha512-yFhIqQAzu2Ca2I4SE2Au3rxVfmohU9Y7wqGR+s7+H7krk26NXhIRAZDgqd6xqjCEFUomDEA3/Bo/7fKmIkW1kA==", + "hasInstallScript": true, + "dependencies": { + "es6-iterator": "^2.0.3", + "es6-symbol": "^3.1.3", + "next-tick": "^1.1.0" + }, + "engines": { + "node": ">=0.10" + } + }, + "node_modules/es6-iterator": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/es6-iterator/-/es6-iterator-2.0.3.tgz", + "integrity": "sha1-p96IkUGgWpSwhUQDstCg+/qY87c=", + "dependencies": { + "d": "1", + "es5-ext": "^0.10.35", + "es6-symbol": "^3.1.1" + } + }, + "node_modules/es6-symbol": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/es6-symbol/-/es6-symbol-3.1.3.tgz", + "integrity": "sha512-NJ6Yn3FuDinBaBRWl/q5X/s4koRHBrgKAu+yGI6JCBeiu3qrcbJhwT2GeR/EXVfylRk8dpQVJoLEFhK+Mu31NA==", + "dependencies": { + "d": "^1.0.1", + "ext": "^1.1.2" + } + }, + "node_modules/es6-weak-map": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/es6-weak-map/-/es6-weak-map-2.0.3.tgz", + "integrity": "sha512-p5um32HOTO1kP+w7PRnB+5lQ43Z6muuMuIMffvDN8ZB4GcnjLBV6zGStpbASIMk4DCAvEaamhe2zhyCb/QXXsA==", + "dependencies": { + "d": "1", + "es5-ext": "^0.10.46", + "es6-iterator": "^2.0.3", + "es6-symbol": "^3.1.1" + } + }, + "node_modules/escalade": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", + "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/escape-goat": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/escape-goat/-/escape-goat-2.1.1.tgz", + "integrity": "sha512-8/uIhbG12Csjy2JEW7D9pHbreaVaS/OpN3ycnyvElTdwM5n6GY6W6e2IPemfvGZeUMqZ9A/3GqIZMgKnBhAw/Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/escape-html": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/escape-html/-/escape-html-1.0.3.tgz", + "integrity": "sha1-Aljq5NPQwJdN4cFpGI7wBR0dGYg=" + }, + "node_modules/escape-string-regexp": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz", + "integrity": "sha1-G2HAViGQqN/2rjuyzwIAyhMLhtQ=", + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/eslint": { + "version": "7.32.0", + "resolved": "https://registry.npmjs.org/eslint/-/eslint-7.32.0.tgz", + "integrity": "sha512-VHZ8gX+EDfz+97jGcgyGCyRia/dPOd6Xh9yPv8Bl1+SoaIwD+a/vlrOmGRUyOYu7MwUhc7CxqeaDZU13S4+EpA==", + "dependencies": { + "@babel/code-frame": "7.12.11", + "@eslint/eslintrc": "^0.4.3", + "@humanwhocodes/config-array": "^0.5.0", + "ajv": "^6.10.0", + "chalk": "^4.0.0", + "cross-spawn": "^7.0.2", + "debug": "^4.0.1", + "doctrine": "^3.0.0", + "enquirer": "^2.3.5", + "escape-string-regexp": "^4.0.0", + "eslint-scope": "^5.1.1", + "eslint-utils": "^2.1.0", + "eslint-visitor-keys": "^2.0.0", + "espree": "^7.3.1", + "esquery": "^1.4.0", + "esutils": "^2.0.2", + "fast-deep-equal": "^3.1.3", + "file-entry-cache": "^6.0.1", + "functional-red-black-tree": "^1.0.1", + "glob-parent": "^5.1.2", + "globals": "^13.6.0", + "ignore": "^4.0.6", + "import-fresh": "^3.0.0", + "imurmurhash": "^0.1.4", + "is-glob": "^4.0.0", + "js-yaml": "^3.13.1", + "json-stable-stringify-without-jsonify": "^1.0.1", + "levn": "^0.4.1", + "lodash.merge": "^4.6.2", + "minimatch": "^3.0.4", + "natural-compare": "^1.4.0", + "optionator": "^0.9.1", + "progress": "^2.0.0", + "regexpp": "^3.1.0", + "semver": "^7.2.1", + "strip-ansi": "^6.0.0", + "strip-json-comments": "^3.1.0", + "table": "^6.0.9", + "text-table": "^0.2.0", + "v8-compile-cache": "^2.0.3" + }, + "bin": { + "eslint": "bin/eslint.js" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/eslint-config-react-app": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/eslint-config-react-app/-/eslint-config-react-app-6.0.0.tgz", + "integrity": "sha512-bpoAAC+YRfzq0dsTk+6v9aHm/uqnDwayNAXleMypGl6CpxI9oXXscVHo4fk3eJPIn+rsbtNetB4r/ZIidFIE8A==", + "dependencies": { + "confusing-browser-globals": "^1.0.10" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependencies": { + "@typescript-eslint/eslint-plugin": "^4.0.0", + "@typescript-eslint/parser": "^4.0.0", + "babel-eslint": "^10.0.0", + "eslint": "^7.5.0", + "eslint-plugin-flowtype": "^5.2.0", + "eslint-plugin-import": "^2.22.0", + "eslint-plugin-jest": "^24.0.0", + "eslint-plugin-jsx-a11y": "^6.3.1", + "eslint-plugin-react": "^7.20.3", + "eslint-plugin-react-hooks": "^4.0.8", + "eslint-plugin-testing-library": "^3.9.0" + }, + "peerDependenciesMeta": { + "eslint-plugin-jest": { + "optional": true + }, + "eslint-plugin-testing-library": { + "optional": true + } + } + }, + "node_modules/eslint-import-resolver-node": { + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.6.tgz", + "integrity": "sha512-0En0w03NRVMn9Uiyn8YRPDKvWjxCWkslUEhGNTdGx15RvPJYQ+lbOlqrlNI2vEAs4pDYK4f/HN2TbDmk5TP0iw==", + "dependencies": { + "debug": "^3.2.7", + "resolve": "^1.20.0" + } + }, + "node_modules/eslint-module-utils": { + "version": "2.7.3", + "resolved": "https://registry.npmjs.org/eslint-module-utils/-/eslint-module-utils-2.7.3.tgz", + "integrity": "sha512-088JEC7O3lDZM9xGe0RerkOMd0EjFl+Yvd1jPWIkMT5u3H9+HC34mWWPnqPrN13gieT9pBOO+Qt07Nb/6TresQ==", + "dependencies": { + "debug": "^3.2.7", + "find-up": "^2.1.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/find-up": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-2.1.0.tgz", + "integrity": "sha1-RdG35QbHF93UgndaK3eSCjwMV6c=", + "dependencies": { + "locate-path": "^2.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/locate-path": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-2.0.0.tgz", + "integrity": "sha1-K1aLJl7slExtnA3pw9u7ygNUzY4=", + "dependencies": { + "p-locate": "^2.0.0", + "path-exists": "^3.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/p-limit": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-1.3.0.tgz", + "integrity": "sha512-vvcXsLAJ9Dr5rQOPk7toZQZJApBl2K4J6dANSsEuh6QI41JYcsS/qhTGa9ErIUUgK3WNQoJYvylxvjqmiqEA9Q==", + "dependencies": { + "p-try": "^1.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/p-locate": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-2.0.0.tgz", + "integrity": "sha1-IKAQOyIqcMj9OcwuWAaA893l7EM=", + "dependencies": { + "p-limit": "^1.1.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/p-try": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-1.0.0.tgz", + "integrity": "sha1-y8ec26+P1CKOE/Yh8rGiN8GyB7M=", + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-module-utils/node_modules/path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha1-zg6+ql94yxiSXqfYENe1mwEP1RU=", + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint-plugin-flowtype": { + "version": "5.10.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-flowtype/-/eslint-plugin-flowtype-5.10.0.tgz", + "integrity": "sha512-vcz32f+7TP+kvTUyMXZmCnNujBQZDNmcqPImw8b9PZ+16w1Qdm6ryRuYZYVaG9xRqqmAPr2Cs9FAX5gN+x/bjw==", + "dependencies": { + "lodash": "^4.17.15", + "string-natural-compare": "^3.0.1" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + }, + "peerDependencies": { + "eslint": "^7.1.0" + } + }, + "node_modules/eslint-plugin-graphql": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-graphql/-/eslint-plugin-graphql-4.0.0.tgz", + "integrity": "sha512-d5tQm24YkVvCEk29ZR5ScsgXqAGCjKlMS8lx3mS7FS/EKsWbkvXQImpvic03EpMIvNTBW5e+2xnHzXB/VHNZJw==", + "dependencies": { + "@babel/runtime": "^7.10.0", + "graphql-config": "^3.0.2", + "lodash.flatten": "^4.4.0", + "lodash.without": "^4.4.0" + }, + "engines": { + "node": ">=10.0" + }, + "peerDependencies": { + "graphql": "^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0" + } + }, + "node_modules/eslint-plugin-import": { + "version": "2.26.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-import/-/eslint-plugin-import-2.26.0.tgz", + "integrity": "sha512-hYfi3FXaM8WPLf4S1cikh/r4IxnO6zrhZbEGz2b660EJRbuxgpDS5gkCuYgGWg2xxh2rBuIr4Pvhve/7c31koA==", + "dependencies": { + "array-includes": "^3.1.4", + "array.prototype.flat": "^1.2.5", + "debug": "^2.6.9", + "doctrine": "^2.1.0", + "eslint-import-resolver-node": "^0.3.6", + "eslint-module-utils": "^2.7.3", + "has": "^1.0.3", + "is-core-module": "^2.8.1", + "is-glob": "^4.0.3", + "minimatch": "^3.1.2", + "object.values": "^1.1.5", + "resolve": "^1.22.0", + "tsconfig-paths": "^3.14.1" + }, + "engines": { + "node": ">=4" + }, + "peerDependencies": { + "eslint": "^2 || ^3 || ^4 || ^5 || ^6 || ^7.2.0 || ^8" + } + }, + "node_modules/eslint-plugin-import/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/eslint-plugin-import/node_modules/doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/eslint-plugin-import/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/eslint-plugin-jsx-a11y": { + "version": "6.5.1", + "resolved": "https://registry.npmjs.org/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.5.1.tgz", + "integrity": "sha512-sVCFKX9fllURnXT2JwLN5Qgo24Ug5NF6dxhkmxsMEUZhXRcGg+X3e1JbJ84YePQKBl5E0ZjAH5Q4rkdcGY99+g==", + "dependencies": { + "@babel/runtime": "^7.16.3", + "aria-query": "^4.2.2", + "array-includes": "^3.1.4", + "ast-types-flow": "^0.0.7", + "axe-core": "^4.3.5", + "axobject-query": "^2.2.0", + "damerau-levenshtein": "^1.0.7", + "emoji-regex": "^9.2.2", + "has": "^1.0.3", + "jsx-ast-utils": "^3.2.1", + "language-tags": "^1.0.5", + "minimatch": "^3.0.4" + }, + "engines": { + "node": ">=4.0" + }, + "peerDependencies": { + "eslint": "^3 || ^4 || ^5 || ^6 || ^7 || ^8" + } + }, + "node_modules/eslint-plugin-jsx-a11y/node_modules/emoji-regex": { + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", + "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==" + }, + "node_modules/eslint-plugin-react": { + "version": "7.29.4", + "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.29.4.tgz", + "integrity": "sha512-CVCXajliVh509PcZYRFyu/BoUEz452+jtQJq2b3Bae4v3xBUWPLCmtmBM+ZinG4MzwmxJgJ2M5rMqhqLVn7MtQ==", + "dependencies": { + "array-includes": "^3.1.4", + "array.prototype.flatmap": "^1.2.5", + "doctrine": "^2.1.0", + "estraverse": "^5.3.0", + "jsx-ast-utils": "^2.4.1 || ^3.0.0", + "minimatch": "^3.1.2", + "object.entries": "^1.1.5", + "object.fromentries": "^2.0.5", + "object.hasown": "^1.1.0", + "object.values": "^1.1.5", + "prop-types": "^15.8.1", + "resolve": "^2.0.0-next.3", + "semver": "^6.3.0", + "string.prototype.matchall": "^4.0.6" + }, + "engines": { + "node": ">=4" + }, + "peerDependencies": { + "eslint": "^3 || ^4 || ^5 || ^6 || ^7 || ^8" + } + }, + "node_modules/eslint-plugin-react-hooks": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.5.0.tgz", + "integrity": "sha512-8k1gRt7D7h03kd+SAAlzXkQwWK22BnK6GKZG+FJA6BAGy22CFvl8kCIXKpVux0cCxMWDQUPqSok0LKaZ0aOcCw==", + "engines": { + "node": ">=10" + }, + "peerDependencies": { + "eslint": "^3.0.0 || ^4.0.0 || ^5.0.0 || ^6.0.0 || ^7.0.0 || ^8.0.0-0" + } + }, + "node_modules/eslint-plugin-react/node_modules/doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/eslint-plugin-react/node_modules/estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==", + "engines": { + "node": ">=4.0" + } + }, + "node_modules/eslint-plugin-react/node_modules/resolve": { + "version": "2.0.0-next.3", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-2.0.0-next.3.tgz", + "integrity": "sha512-W8LucSynKUIDu9ylraa7ueVZ7hc0uAgJBxVsQSKOXOyle8a93qXhcz+XAXZ8bIq2d6i4Ehddn6Evt+0/UwKk6Q==", + "dependencies": { + "is-core-module": "^2.2.0", + "path-parse": "^1.0.6" + } + }, + "node_modules/eslint-plugin-react/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/eslint-scope": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-5.1.1.tgz", + "integrity": "sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw==", + "dependencies": { + "esrecurse": "^4.3.0", + "estraverse": "^4.1.1" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/eslint-utils": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/eslint-utils/-/eslint-utils-3.0.0.tgz", + "integrity": "sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA==", + "dependencies": { + "eslint-visitor-keys": "^2.0.0" + }, + "engines": { + "node": "^10.0.0 || ^12.0.0 || >= 14.0.0" + }, + "peerDependencies": { + "eslint": ">=5" + } + }, + "node_modules/eslint-visitor-keys": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-2.1.0.tgz", + "integrity": "sha512-0rSmRBzXgDzIsD6mGdJgevzgezI534Cer5L/vyMX0kHzT/jiB43jRhd9YUlMGYLQy2zprNmoT8qasCGtY+QaKw==", + "engines": { + "node": ">=10" + } + }, + "node_modules/eslint-webpack-plugin": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/eslint-webpack-plugin/-/eslint-webpack-plugin-2.6.0.tgz", + "integrity": "sha512-V+LPY/T3kur5QO3u+1s34VDTcRxjXWPUGM4hlmTb5DwVD0OQz631yGTxJZf4SpAqAjdbBVe978S8BJeHpAdOhQ==", + "dependencies": { + "@types/eslint": "^7.28.2", + "arrify": "^2.0.1", + "jest-worker": "^27.3.1", + "micromatch": "^4.0.4", + "normalize-path": "^3.0.0", + "schema-utils": "^3.1.1" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "eslint": "^7.0.0 || ^8.0.0", + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/eslint-webpack-plugin/node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/eslint-webpack-plugin/node_modules/jest-worker": { + "version": "27.5.1", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-27.5.1.tgz", + "integrity": "sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg==", + "dependencies": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^8.0.0" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/eslint-webpack-plugin/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/eslint-webpack-plugin/node_modules/supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/eslint/node_modules/@babel/code-frame": { + "version": "7.12.11", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.12.11.tgz", + "integrity": "sha512-Zt1yodBx1UcyiePMSkWnU4hPqhwq7hGi2nFL1LeA3EUl+q2LQx16MISgJ0+z7dnmgvP9QtIleuETGOiOH1RcIw==", + "dependencies": { + "@babel/highlight": "^7.10.4" + } + }, + "node_modules/eslint/node_modules/cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "dependencies": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/eslint/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/eslint/node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/eslint/node_modules/eslint-utils": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/eslint-utils/-/eslint-utils-2.1.0.tgz", + "integrity": "sha512-w94dQYoauyvlDc43XnGB8lU3Zt713vNChgt4EWwhXAP2XkBvndfxF0AgIqKOOasjPIPzj9JqgwkwbCYD0/V3Zg==", + "dependencies": { + "eslint-visitor-keys": "^1.1.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/eslint/node_modules/eslint-utils/node_modules/eslint-visitor-keys": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz", + "integrity": "sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/eslint/node_modules/globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "dependencies": { + "type-fest": "^0.20.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/eslint/node_modules/ignore": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-4.0.6.tgz", + "integrity": "sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg==", + "engines": { + "node": ">= 4" + } + }, + "node_modules/eslint/node_modules/path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/eslint/node_modules/shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "dependencies": { + "shebang-regex": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/eslint/node_modules/shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", + "engines": { + "node": ">=8" + } + }, + "node_modules/eslint/node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/espree": { + "version": "7.3.1", + "resolved": "https://registry.npmjs.org/espree/-/espree-7.3.1.tgz", + "integrity": "sha512-v3JCNCE64umkFpmkFGqzVKsOT0tN1Zr+ueqLZfpV1Ob8e+CEgPWa+OxCoGH3tnhimMKIaBm4m/vaRpJ/krRz2g==", + "dependencies": { + "acorn": "^7.4.0", + "acorn-jsx": "^5.3.1", + "eslint-visitor-keys": "^1.3.0" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/espree/node_modules/eslint-visitor-keys": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz", + "integrity": "sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/esprima": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/esprima/-/esprima-4.0.1.tgz", + "integrity": "sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A==", + "bin": { + "esparse": "bin/esparse.js", + "esvalidate": "bin/esvalidate.js" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/esquery": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.4.0.tgz", + "integrity": "sha512-cCDispWt5vHHtwMY2YrAQ4ibFkAL8RbH5YGBnZBc90MolvvfkkQcJro/aZiAQUlQ3qgrYS6D6v8Gc5G5CQsc9w==", + "dependencies": { + "estraverse": "^5.1.0" + }, + "engines": { + "node": ">=0.10" + } + }, + "node_modules/esquery/node_modules/estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==", + "engines": { + "node": ">=4.0" + } + }, + "node_modules/esrecurse": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/esrecurse/-/esrecurse-4.3.0.tgz", + "integrity": "sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag==", + "dependencies": { + "estraverse": "^5.2.0" + }, + "engines": { + "node": ">=4.0" + } + }, + "node_modules/esrecurse/node_modules/estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==", + "engines": { + "node": ">=4.0" + } + }, + "node_modules/estraverse": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-4.3.0.tgz", + "integrity": "sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw==", + "engines": { + "node": ">=4.0" + } + }, + "node_modules/esutils": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/esutils/-/esutils-2.0.3.tgz", + "integrity": "sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/etag": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/etag/-/etag-1.8.1.tgz", + "integrity": "sha1-Qa4u62XvpiJorr/qg6x9eSmbCIc=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/event-emitter": { + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/event-emitter/-/event-emitter-0.3.5.tgz", + "integrity": "sha1-34xp7vFkeSPHFXuc6DhAYQsCzDk=", + "dependencies": { + "d": "1", + "es5-ext": "~0.10.14" + } + }, + "node_modules/event-source-polyfill": { + "version": "1.0.25", + "resolved": "https://registry.npmjs.org/event-source-polyfill/-/event-source-polyfill-1.0.25.tgz", + "integrity": "sha512-hQxu6sN1Eq4JjoI7ITdQeGGUN193A2ra83qC0Ltm9I2UJVAten3OFVN6k5RX4YWeCS0BoC8xg/5czOCIHVosQg==" + }, + "node_modules/event-target-shim": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/event-target-shim/-/event-target-shim-5.0.1.tgz", + "integrity": "sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/eventemitter3": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-3.1.2.tgz", + "integrity": "sha512-tvtQIeLVHjDkJYnzf2dgVMxfuSGJeM/7UCG17TT4EumTfNtF+0nebF/4zWOIkCreAbtNqhGEboB6BWrwqNaw4Q==" + }, + "node_modules/events": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/events/-/events-3.3.0.tgz", + "integrity": "sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==", + "engines": { + "node": ">=0.8.x" + } + }, + "node_modules/execa": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", + "integrity": "sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg==", + "dependencies": { + "cross-spawn": "^7.0.3", + "get-stream": "^6.0.0", + "human-signals": "^2.1.0", + "is-stream": "^2.0.0", + "merge-stream": "^2.0.0", + "npm-run-path": "^4.0.1", + "onetime": "^5.1.2", + "signal-exit": "^3.0.3", + "strip-final-newline": "^2.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/execa/node_modules/cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "dependencies": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/execa/node_modules/get-stream": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-6.0.1.tgz", + "integrity": "sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg==", + "engines": { + "node": ">=10" + } + }, + "node_modules/execa/node_modules/path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/execa/node_modules/shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "dependencies": { + "shebang-regex": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/execa/node_modules/shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", + "engines": { + "node": ">=8" + } + }, + "node_modules/execa/node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/exif-parser": { + "version": "0.1.12", + "resolved": "https://registry.npmjs.org/exif-parser/-/exif-parser-0.1.12.tgz", + "integrity": "sha1-WKnS1ywCwfbwKg70qRZicrd2CSI=" + }, + "node_modules/expand-template": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/expand-template/-/expand-template-2.0.3.tgz", + "integrity": "sha512-XYfuKMvj4O35f/pOXLObndIRvyQ+/+6AhODh+OKWj9S9498pHHn/IMszH+gt0fBCRWMNfk1ZSp5x3AifmnI2vg==", + "engines": { + "node": ">=6" + } + }, + "node_modules/express": { + "version": "4.18.1", + "resolved": "https://registry.npmjs.org/express/-/express-4.18.1.tgz", + "integrity": "sha512-zZBcOX9TfehHQhtupq57OF8lFZ3UZi08Y97dwFCkD8p9d/d2Y3M+ykKcwaMDEL+4qyUolgBDX6AblpR3fL212Q==", + "dependencies": { + "accepts": "~1.3.8", + "array-flatten": "1.1.1", + "body-parser": "1.20.0", + "content-disposition": "0.5.4", + "content-type": "~1.0.4", + "cookie": "0.5.0", + "cookie-signature": "1.0.6", + "debug": "2.6.9", + "depd": "2.0.0", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "etag": "~1.8.1", + "finalhandler": "1.2.0", + "fresh": "0.5.2", + "http-errors": "2.0.0", + "merge-descriptors": "1.0.1", + "methods": "~1.1.2", + "on-finished": "2.4.1", + "parseurl": "~1.3.3", + "path-to-regexp": "0.1.7", + "proxy-addr": "~2.0.7", + "qs": "6.10.3", + "range-parser": "~1.2.1", + "safe-buffer": "5.2.1", + "send": "0.18.0", + "serve-static": "1.15.0", + "setprototypeof": "1.2.0", + "statuses": "2.0.1", + "type-is": "~1.6.18", + "utils-merge": "1.0.1", + "vary": "~1.1.2" + }, + "engines": { + "node": ">= 0.10.0" + } + }, + "node_modules/express-graphql": { + "version": "0.12.0", + "resolved": "https://registry.npmjs.org/express-graphql/-/express-graphql-0.12.0.tgz", + "integrity": "sha512-DwYaJQy0amdy3pgNtiTDuGGM2BLdj+YO2SgbKoLliCfuHv3VVTt7vNG/ZqK2hRYjtYHE2t2KB705EU94mE64zg==", + "dependencies": { + "accepts": "^1.3.7", + "content-type": "^1.0.4", + "http-errors": "1.8.0", + "raw-body": "^2.4.1" + }, + "engines": { + "node": ">= 10.x" + }, + "peerDependencies": { + "graphql": "^14.7.0 || ^15.3.0" + } + }, + "node_modules/express-graphql/node_modules/depd": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/depd/-/depd-1.1.2.tgz", + "integrity": "sha1-m81S4UwJd2PnSbJ0xDRu0uVgtak=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/express-graphql/node_modules/http-errors": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/http-errors/-/http-errors-1.8.0.tgz", + "integrity": "sha512-4I8r0C5JDhT5VkvI47QktDW75rNlGVsUf/8hzjCC/wkWI/jdTRmBb9aI7erSG82r1bjKY3F6k28WnsVxB1C73A==", + "dependencies": { + "depd": "~1.1.2", + "inherits": "2.0.4", + "setprototypeof": "1.2.0", + "statuses": ">= 1.5.0 < 2", + "toidentifier": "1.0.0" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/express-graphql/node_modules/statuses": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/statuses/-/statuses-1.5.0.tgz", + "integrity": "sha1-Fhx9rBd2Wf2YEfQ3cfqZOBR4Yow=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/express-graphql/node_modules/toidentifier": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/toidentifier/-/toidentifier-1.0.0.tgz", + "integrity": "sha512-yaOH/Pk/VEhBWWTlhI+qXxDFXlejDGcQipMlyxda9nthulaxLZUNcUqFxokp0vcYnvteJln5FNQDRrxj3YcbVw==", + "engines": { + "node": ">=0.6" + } + }, + "node_modules/express/node_modules/cookie": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.5.0.tgz", + "integrity": "sha512-YZ3GUyn/o8gfKJlnlX7g7xq4gyO6OSuhGPKaaGssGB2qgDUS0gPgtTvoyZLTt9Ab6dC4hfc9dV5arkvc/OCmrw==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/express/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/express/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/express/node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + }, + "node_modules/ext": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/ext/-/ext-1.6.0.tgz", + "integrity": "sha512-sdBImtzkq2HpkdRLtlLWDa6w4DX22ijZLKx8BMPUuKe1c5lbN6xwQDQCxSfxBQnHZ13ls/FH0MQZx/q/gr6FQg==", + "dependencies": { + "type": "^2.5.0" + } + }, + "node_modules/ext/node_modules/type": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/type/-/type-2.6.0.tgz", + "integrity": "sha512-eiDBDOmkih5pMbo9OqsqPRGMljLodLcwd5XD5JbtNB0o89xZAwynY9EdCDsJU7LtcVCClu9DvM7/0Ep1hYX3EQ==" + }, + "node_modules/extend": { + "version": "3.0.2", + "resolved": "https://registry.npmmirror.com/extend/-/extend-3.0.2.tgz", + "integrity": "sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==" + }, + "node_modules/extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmmirror.com/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha512-zCnTtlxNoAiDc3gqY2aYAWFx7XWWiasuF2K8Me5WbN8otHKTUKBwjPtNpRs/rbUZm7KxWAaNj7P1a/p52GbVug==", + "dependencies": { + "is-extendable": "^0.1.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/external-editor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/external-editor/-/external-editor-3.1.0.tgz", + "integrity": "sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew==", + "dependencies": { + "chardet": "^0.7.0", + "iconv-lite": "^0.4.24", + "tmp": "^0.0.33" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/external-editor/node_modules/tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==", + "dependencies": { + "os-tmpdir": "~1.0.2" + }, + "engines": { + "node": ">=0.6.0" + } + }, + "node_modules/extract-files": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/extract-files/-/extract-files-9.0.0.tgz", + "integrity": "sha512-CvdFfHkC95B4bBBk36hcEmvdR2awOdhhVUYH6S/zrVj3477zven/fJMYg7121h4T1xHZC+tetUpubpAhxwI7hQ==", + "engines": { + "node": "^10.17.0 || ^12.0.0 || >= 13.7.0" + } + }, + "node_modules/extsprintf": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/extsprintf/-/extsprintf-1.3.0.tgz", + "integrity": "sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==", + "engines": [ + "node >=0.6.0" + ] + }, + "node_modules/fast-deep-equal": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz", + "integrity": "sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q==" + }, + "node_modules/fast-glob": { + "version": "3.2.11", + "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-3.2.11.tgz", + "integrity": "sha512-xrO3+1bxSo3ZVHAnqzyuewYT6aMFHRAd4Kcs92MAonjwQZLsK9d0SF1IyQ3k5PoirxTW0Oe/RqFgMQ6TcNE5Ew==", + "dependencies": { + "@nodelib/fs.stat": "^2.0.2", + "@nodelib/fs.walk": "^1.2.3", + "glob-parent": "^5.1.2", + "merge2": "^1.3.0", + "micromatch": "^4.0.4" + }, + "engines": { + "node": ">=8.6.0" + } + }, + "node_modules/fast-json-stable-stringify": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz", + "integrity": "sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw==" + }, + "node_modules/fast-levenshtein": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz", + "integrity": "sha1-PYpcZog6FqMMqGQ+hR8Zuqd5eRc=" + }, + "node_modules/fastest-levenshtein": { + "version": "1.0.12", + "resolved": "https://registry.npmjs.org/fastest-levenshtein/-/fastest-levenshtein-1.0.12.tgz", + "integrity": "sha512-On2N+BpYJ15xIC974QNVuYGMOlEVt4s0EOI3wwMqOmK1fdDY+FN/zltPV8vosq4ad4c/gJ1KHScUn/6AWIgiow==" + }, + "node_modules/fastq": { + "version": "1.13.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.13.0.tgz", + "integrity": "sha512-YpkpUnK8od0o1hmeSc7UUs/eB/vIPWJYjKck2QKIzAf71Vm1AAQ3EbuZB3g2JIy+pg+ERD0vqI79KyZiB2e2Nw==", + "dependencies": { + "reusify": "^1.0.4" + } + }, + "node_modules/fb-watchman": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/fb-watchman/-/fb-watchman-2.0.1.tgz", + "integrity": "sha512-DkPJKQeY6kKwmuMretBhr7G6Vodr7bFwDYTXIkfG1gjvNpaxBTQV3PbXg6bR1c1UP4jPOX0jHUbbHANL9vRjVg==", + "dependencies": { + "bser": "2.1.1" + } + }, + "node_modules/fbjs": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/fbjs/-/fbjs-3.0.4.tgz", + "integrity": "sha512-ucV0tDODnGV3JCnnkmoszb5lf4bNpzjv80K41wd4k798Etq+UYD0y0TIfalLjZoKgjive6/adkRnszwapiDgBQ==", + "dependencies": { + "cross-fetch": "^3.1.5", + "fbjs-css-vars": "^1.0.0", + "loose-envify": "^1.0.0", + "object-assign": "^4.1.0", + "promise": "^7.1.1", + "setimmediate": "^1.0.5", + "ua-parser-js": "^0.7.30" + } + }, + "node_modules/fbjs-css-vars": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/fbjs-css-vars/-/fbjs-css-vars-1.0.2.tgz", + "integrity": "sha512-b2XGFAFdWZWg0phtAWLHCk836A1Xann+I+Dgd3Gk64MHKZO44FfoD1KxyvbSh0qZsIoXQGGlVztIY+oitJPpRQ==" + }, + "node_modules/fd": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/fd/-/fd-0.0.3.tgz", + "integrity": "sha512-iAHrIslQb3U68OcMSP0kkNWabp7sSN6d2TBSb2JO3gcLJVDd4owr/hKM4SFJovFOUeeXeItjYgouEDTMWiVAnA==" + }, + "node_modules/figures": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/figures/-/figures-3.2.0.tgz", + "integrity": "sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg==", + "dependencies": { + "escape-string-regexp": "^1.0.5" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/file-entry-cache": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-6.0.1.tgz", + "integrity": "sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg==", + "dependencies": { + "flat-cache": "^3.0.4" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/file-loader": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/file-loader/-/file-loader-6.2.0.tgz", + "integrity": "sha512-qo3glqyTa61Ytg4u73GultjHGjdRyig3tG6lPtyX/jOEJvHif9uB0/OCI2Kif6ctF3caQTW2G5gym21oAsI4pw==", + "dependencies": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/file-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/file-type": { + "version": "16.5.3", + "resolved": "https://registry.npmjs.org/file-type/-/file-type-16.5.3.tgz", + "integrity": "sha512-uVsl7iFhHSOY4bEONLlTK47iAHtNsFHWP5YE4xJfZ4rnX7S1Q3wce09XgqSC7E/xh8Ncv/be1lNoyprlUH/x6A==", + "dependencies": { + "readable-web-to-node-stream": "^3.0.0", + "strtok3": "^6.2.4", + "token-types": "^4.1.1" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/filename-reserved-regex": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/filename-reserved-regex/-/filename-reserved-regex-2.0.0.tgz", + "integrity": "sha1-q/c9+rc10EVECr/qLZHzieu/oik=", + "engines": { + "node": ">=4" + } + }, + "node_modules/filenamify": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/filenamify/-/filenamify-4.3.0.tgz", + "integrity": "sha512-hcFKyUG57yWGAzu1CMt/dPzYZuv+jAJUT85bL8mrXvNe6hWj6yEHEc4EdcgiA6Z3oi1/9wXJdZPXF2dZNgwgOg==", + "dependencies": { + "filename-reserved-regex": "^2.0.0", + "strip-outer": "^1.0.1", + "trim-repeated": "^1.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/filesize": { + "version": "8.0.7", + "resolved": "https://registry.npmjs.org/filesize/-/filesize-8.0.7.tgz", + "integrity": "sha512-pjmC+bkIF8XI7fWaH8KxHcZL3DPybs1roSKP4rKDvy20tAWwIObE4+JIseG2byfGKhud5ZnM4YSGKBz7Sh0ndQ==", + "engines": { + "node": ">= 0.4.0" + } + }, + "node_modules/fill-range": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", + "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "dependencies": { + "to-regex-range": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/filter-obj": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/filter-obj/-/filter-obj-1.1.0.tgz", + "integrity": "sha1-mzERErxsYSehbgFsbF1/GeCAXFs=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/finalhandler": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.2.0.tgz", + "integrity": "sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg==", + "dependencies": { + "debug": "2.6.9", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "on-finished": "2.4.1", + "parseurl": "~1.3.3", + "statuses": "2.0.1", + "unpipe": "~1.0.0" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/finalhandler/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/finalhandler/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/find-cache-dir": { + "version": "3.3.2", + "resolved": "https://registry.npmjs.org/find-cache-dir/-/find-cache-dir-3.3.2.tgz", + "integrity": "sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig==", + "dependencies": { + "commondir": "^1.0.1", + "make-dir": "^3.0.2", + "pkg-dir": "^4.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/find-up": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", + "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "dependencies": { + "locate-path": "^5.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/flat-cache": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-3.0.4.tgz", + "integrity": "sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg==", + "dependencies": { + "flatted": "^3.1.0", + "rimraf": "^3.0.2" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/flatted": { + "version": "3.2.5", + "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.2.5.tgz", + "integrity": "sha512-WIWGi2L3DyTUvUrwRKgGi9TwxQMUEqPOPQBVi71R96jZXJdFskXEmf54BoZaS1kknGODoIGASGEzBUYdyMCBJg==" + }, + "node_modules/follow-redirects": { + "version": "1.15.0", + "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.0.tgz", + "integrity": "sha512-aExlJShTV4qOUOL7yF1U5tvLCB0xQuudbf6toyYA0E/acBNw71mvjFTnLaRp50aQaYocMR0a/RMMBIHeZnGyjQ==", + "engines": { + "node": ">=4.0" + }, + "peerDependenciesMeta": { + "debug": { + "optional": true + } + } + }, + "node_modules/forever-agent": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/forever-agent/-/forever-agent-0.6.1.tgz", + "integrity": "sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==", + "engines": { + "node": "*" + } + }, + "node_modules/fork-ts-checker-webpack-plugin": { + "version": "6.5.2", + "resolved": "https://registry.npmjs.org/fork-ts-checker-webpack-plugin/-/fork-ts-checker-webpack-plugin-6.5.2.tgz", + "integrity": "sha512-m5cUmF30xkZ7h4tWUgTAcEaKmUW7tfyUyTqNNOz7OxWJ0v1VWKTcOvH8FWHUwSjlW/356Ijc9vi3XfcPstpQKA==", + "dependencies": { + "@babel/code-frame": "^7.8.3", + "@types/json-schema": "^7.0.5", + "chalk": "^4.1.0", + "chokidar": "^3.4.2", + "cosmiconfig": "^6.0.0", + "deepmerge": "^4.2.2", + "fs-extra": "^9.0.0", + "glob": "^7.1.6", + "memfs": "^3.1.2", + "minimatch": "^3.0.4", + "schema-utils": "2.7.0", + "semver": "^7.3.2", + "tapable": "^1.0.0" + }, + "engines": { + "node": ">=10", + "yarn": ">=1.0.0" + }, + "peerDependencies": { + "eslint": ">= 6", + "typescript": ">= 2.7", + "vue-template-compiler": "*", + "webpack": ">= 4" + }, + "peerDependenciesMeta": { + "eslint": { + "optional": true + }, + "vue-template-compiler": { + "optional": true + } + } + }, + "node_modules/fork-ts-checker-webpack-plugin/node_modules/cosmiconfig": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-6.0.0.tgz", + "integrity": "sha512-xb3ZL6+L8b9JLLCx3ZdoZy4+2ECphCMo2PwqgP1tlfVq6M6YReyzBJtvWWtbDSpNr9hn96pkCiZqUcFEc+54Qg==", + "dependencies": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.1.0", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.7.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/fork-ts-checker-webpack-plugin/node_modules/fs-extra": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-9.1.0.tgz", + "integrity": "sha512-hcg3ZmepS30/7BSFqRvoo3DOMQu7IjqxO5nCDt+zM9XWjb33Wg7ziNT+Qvqbuc3+gWpzO02JubVyk2G4Zvo1OQ==", + "dependencies": { + "at-least-node": "^1.0.0", + "graceful-fs": "^4.2.0", + "jsonfile": "^6.0.1", + "universalify": "^2.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/fork-ts-checker-webpack-plugin/node_modules/schema-utils": { + "version": "2.7.0", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-2.7.0.tgz", + "integrity": "sha512-0ilKFI6QQF5nxDZLFn2dMjvc4hjg/Wkg7rHd3jK6/A4a1Hl9VFdQWvgB1UMGoU94pad1P/8N7fMcEnLnSiju8A==", + "dependencies": { + "@types/json-schema": "^7.0.4", + "ajv": "^6.12.2", + "ajv-keywords": "^3.4.1" + }, + "engines": { + "node": ">= 8.9.0" + } + }, + "node_modules/fork-ts-checker-webpack-plugin/node_modules/tapable": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-1.1.3.tgz", + "integrity": "sha512-4WK/bYZmj8xLr+HUCODHGF1ZFzsYffasLUgEiMBY4fgtltdO6B4WJtlSbPaDTLpYTcGVwM2qLnFTICEcNxs3kA==", + "engines": { + "node": ">=6" + } + }, + "node_modules/form-data": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", + "integrity": "sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/forwarded": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/forwarded/-/forwarded-0.2.0.tgz", + "integrity": "sha512-buRG0fpBtRHSTCOASe6hD258tEubFoRLb4ZNA6NxMVHNw2gOcwHo9wyablzMzOA5z9xA9L1KNjk/Nt6MT9aYow==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/fraction.js": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/fraction.js/-/fraction.js-4.2.0.tgz", + "integrity": "sha512-MhLuK+2gUcnZe8ZHlaaINnQLl0xRIGRfcGk2yl8xoQAfHrSsL3rYu6FCmBdkdbhc9EPlwyGHewaRsvwRMJtAlA==", + "engines": { + "node": "*" + } + }, + "node_modules/fresh": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/fresh/-/fresh-0.5.2.tgz", + "integrity": "sha1-PYyt2Q2XZWn6g1qx+OSyOhBWBac=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/fs-constants": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs-constants/-/fs-constants-1.0.0.tgz", + "integrity": "sha512-y6OAwoSIf7FyjMIv94u+b5rdheZEjzR63GTyZJm5qh4Bi+2YgwLCcI/fPFZkL5PSixOt6ZNKm+w+Hfp/Bciwow==" + }, + "node_modules/fs-exists-cached": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs-exists-cached/-/fs-exists-cached-1.0.0.tgz", + "integrity": "sha1-zyVVTKBQ3EmuZla0HeQiWJidy84=" + }, + "node_modules/fs-extra": { + "version": "10.1.0", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-10.1.0.tgz", + "integrity": "sha512-oRXApq54ETRj4eMiFzGnHWGy+zo5raudjuxN0b8H7s/RU2oW0Wvsx9O0ACRN/kRq9E8Vu/ReskGB5o3ji+FzHQ==", + "dependencies": { + "graceful-fs": "^4.2.0", + "jsonfile": "^6.0.1", + "universalify": "^2.0.0" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/fs-minipass": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/fs-minipass/-/fs-minipass-2.1.0.tgz", + "integrity": "sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg==", + "dependencies": { + "minipass": "^3.0.0" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/fs-monkey": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/fs-monkey/-/fs-monkey-1.0.3.tgz", + "integrity": "sha512-cybjIfiiE+pTWicSCLFHSrXZ6EilF30oh91FDP9S2B051prEa7QWfrVTQm10/dDpswBDXZugPa1Ogu8Yh+HV0Q==" + }, + "node_modules/fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=" + }, + "node_modules/fsevents": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.2.tgz", + "integrity": "sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA==", + "hasInstallScript": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": "^8.16.0 || ^10.6.0 || >=11.0.0" + } + }, + "node_modules/function-bind": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz", + "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==" + }, + "node_modules/function.prototype.name": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/function.prototype.name/-/function.prototype.name-1.1.5.tgz", + "integrity": "sha512-uN7m/BzVKQnCUF/iW8jYea67v++2u7m5UgENbHRtdDVclOUP+FMPlCNdmk0h/ysGyo2tavMJEDqJAkJdRa1vMA==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.0", + "functions-have-names": "^1.2.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/functional-red-black-tree": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz", + "integrity": "sha1-GwqzvVU7Kg1jmdKcDj6gslIHgyc=" + }, + "node_modules/functions-have-names": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/functions-have-names/-/functions-have-names-1.2.3.tgz", + "integrity": "sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==" + }, + "node_modules/gatsby": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby/-/gatsby-4.14.0.tgz", + "integrity": "sha512-7Nwy+1QHligFcV94I/Gvs6rBHu+GN5L2s2FoSU2h0vs+p3ys0UzpBhaqxqiPZynfCWX3vRUquDvoeSR1ALYgOg==", + "hasInstallScript": true, + "dependencies": { + "@babel/code-frame": "^7.14.0", + "@babel/core": "^7.15.5", + "@babel/eslint-parser": "^7.15.4", + "@babel/helper-plugin-utils": "^7.14.5", + "@babel/parser": "^7.15.5", + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "@babel/types": "^7.15.4", + "@gatsbyjs/reach-router": "^1.3.6", + "@gatsbyjs/webpack-hot-middleware": "^2.25.2", + "@graphql-codegen/add": "^3.1.1", + "@graphql-codegen/core": "^2.5.1", + "@graphql-codegen/plugin-helpers": "^2.4.2", + "@graphql-codegen/typescript": "^2.4.8", + "@graphql-codegen/typescript-operations": "^2.3.5", + "@graphql-tools/code-file-loader": "^7.2.14", + "@graphql-tools/load": "^7.5.10", + "@nodelib/fs.walk": "^1.2.8", + "@parcel/core": "^2.3.2", + "@pmmmwh/react-refresh-webpack-plugin": "^0.4.3", + "@types/http-proxy": "^1.17.7", + "@typescript-eslint/eslint-plugin": "^4.33.0", + "@typescript-eslint/parser": "^4.33.0", + "@vercel/webpack-asset-relocator-loader": "^1.7.0", + "address": "1.1.2", + "anser": "^2.1.0", + "autoprefixer": "^10.4.0", + "axios": "^0.21.1", + "babel-loader": "^8.2.3", + "babel-plugin-add-module-exports": "^1.0.4", + "babel-plugin-dynamic-import-node": "^2.3.3", + "babel-plugin-lodash": "^3.3.4", + "babel-plugin-remove-graphql-queries": "^4.14.0", + "babel-preset-gatsby": "^2.14.0", + "better-opn": "^2.1.1", + "bluebird": "^3.7.2", + "body-parser": "^1.19.0", + "browserslist": "^4.17.5", + "cache-manager": "^2.11.1", + "chalk": "^4.1.2", + "chokidar": "^3.5.2", + "common-tags": "^1.8.0", + "compression": "^1.7.4", + "cookie": "^0.4.1", + "core-js": "^3.22.3", + "cors": "^2.8.5", + "css-loader": "^5.2.7", + "css-minimizer-webpack-plugin": "^2.0.0", + "css.escape": "^1.5.1", + "date-fns": "^2.25.0", + "debug": "^3.2.7", + "deepmerge": "^4.2.2", + "detect-port": "^1.3.0", + "devcert": "^1.2.0", + "dotenv": "^8.6.0", + "enhanced-resolve": "^5.8.3", + "eslint": "^7.32.0", + "eslint-config-react-app": "^6.0.0", + "eslint-plugin-flowtype": "^5.10.0", + "eslint-plugin-graphql": "^4.0.0", + "eslint-plugin-import": "^2.26.0", + "eslint-plugin-jsx-a11y": "^6.5.1", + "eslint-plugin-react": "^7.29.4", + "eslint-plugin-react-hooks": "^4.5.0", + "eslint-webpack-plugin": "^2.6.0", + "event-source-polyfill": "1.0.25", + "execa": "^5.1.1", + "express": "^4.17.1", + "express-graphql": "^0.12.0", + "fastest-levenshtein": "^1.0.12", + "fastq": "^1.13.0", + "file-loader": "^6.2.0", + "find-cache-dir": "^3.3.2", + "fs-exists-cached": "1.0.0", + "fs-extra": "^10.1.0", + "gatsby-cli": "^4.14.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-graphiql-explorer": "^2.14.0", + "gatsby-legacy-polyfills": "^2.14.0", + "gatsby-link": "^4.14.0", + "gatsby-page-utils": "^2.14.0", + "gatsby-parcel-config": "^0.5.0", + "gatsby-plugin-page-creator": "^4.14.0", + "gatsby-plugin-typescript": "^4.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-react-router-scroll": "^5.14.0", + "gatsby-telemetry": "^3.14.0", + "gatsby-worker": "^1.14.0", + "glob": "^7.2.0", + "globby": "^11.1.0", + "got": "^11.8.2", + "graphql": "^15.7.2", + "graphql-compose": "^9.0.7", + "graphql-playground-middleware-express": "^1.7.22", + "hasha": "^5.2.2", + "http-proxy": "^1.18.1", + "invariant": "^2.2.4", + "is-relative": "^1.0.0", + "is-relative-url": "^3.0.0", + "joi": "^17.4.2", + "json-loader": "^0.5.7", + "latest-version": "5.1.0", + "lmdb": "~2.2.3", + "lodash": "^4.17.21", + "md5-file": "^5.0.0", + "meant": "^1.0.3", + "memoizee": "^0.4.15", + "micromatch": "^4.0.4", + "mime": "^2.5.2", + "mini-css-extract-plugin": "1.6.2", + "mitt": "^1.2.0", + "moment": "^2.29.1", + "multer": "^1.4.3", + "node-fetch": "^2.6.6", + "normalize-path": "^3.0.0", + "null-loader": "^4.0.1", + "opentracing": "^0.14.5", + "p-defer": "^3.0.0", + "parseurl": "^1.3.3", + "physical-cpu-count": "^2.0.0", + "platform": "^1.3.6", + "postcss": "^8.3.11", + "postcss-flexbugs-fixes": "^5.0.2", + "postcss-loader": "^5.3.0", + "prompts": "^2.4.2", + "prop-types": "^15.7.2", + "query-string": "^6.14.1", + "raw-loader": "^4.0.2", + "react-dev-utils": "^12.0.1", + "react-refresh": "^0.9.0", + "redux": "4.1.2", + "redux-thunk": "^2.4.0", + "resolve-from": "^5.0.0", + "semver": "^7.3.7", + "shallow-compare": "^1.2.2", + "signal-exit": "^3.0.5", + "slugify": "^1.6.1", + "socket.io": "3.1.2", + "socket.io-client": "3.1.3", + "source-map": "^0.7.3", + "source-map-support": "^0.5.20", + "st": "^2.0.0", + "stack-trace": "^0.0.10", + "string-similarity": "^1.2.2", + "strip-ansi": "^6.0.1", + "style-loader": "^2.0.0", + "terser-webpack-plugin": "^5.2.4", + "tmp": "^0.2.1", + "true-case-path": "^2.2.1", + "type-of": "^2.0.1", + "url-loader": "^4.1.1", + "uuid": "^8.3.2", + "webpack": "^5.61.0", + "webpack-dev-middleware": "^4.3.0", + "webpack-merge": "^5.8.0", + "webpack-stats-plugin": "^1.0.3", + "webpack-virtual-modules": "^0.3.2", + "xstate": "^4.26.0", + "yaml-loader": "^0.6.0" + }, + "bin": { + "gatsby": "cli.js" + }, + "engines": { + "node": ">=14.15.0" + }, + "optionalDependencies": { + "gatsby-sharp": "^0.8.0" + }, + "peerDependencies": { + "react": "^16.9.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.9.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/gatsby-core-utils": { + "version": "3.14.0", + "resolved": "https://registry.npmjs.org/gatsby-core-utils/-/gatsby-core-utils-3.14.0.tgz", + "integrity": "sha512-JavHwcX5L+ZRoL5FKhYex3JfbwwS0273YTpf8y8SRKsObD8H+bbLOUlbOjASpqy+IU3dW+r76gT1dQdaqeH9Og==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "ci-info": "2.0.0", + "configstore": "^5.0.1", + "fastq": "^1.13.0", + "file-type": "^16.5.3", + "fs-extra": "^10.1.0", + "got": "^11.8.3", + "import-from": "^4.0.0", + "lmdb": "^2.2.6", + "lock": "^1.1.0", + "node-object-hash": "^2.3.10", + "proper-lockfile": "^4.1.2", + "resolve-from": "^5.0.0", + "tmp": "^0.2.1", + "xdg-basedir": "^4.0.0" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby-graphiql-explorer": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-graphiql-explorer/-/gatsby-graphiql-explorer-2.14.0.tgz", + "integrity": "sha512-J71G+WtSRmykmmdqYYGz5CYC6zToTmJqyywKpN83aZF2z7h7Ab2FHBuiP84KIlF2xpSxsk26puZ40TIHOGP2yw==", + "dependencies": { + "@babel/runtime": "^7.15.4" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby-legacy-polyfills": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-legacy-polyfills/-/gatsby-legacy-polyfills-2.14.0.tgz", + "integrity": "sha512-OcJrY9eqiHtU8bi1zOiaO+wXZv+W/HOR0oP+5IvmWBIiLl4M+ln/z6PJcqk2fnfIK51zyzARvhPXAakDs5JE4w==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "core-js-compat": "3.9.0" + } + }, + "node_modules/gatsby-legacy-polyfills/node_modules/core-js-compat": { + "version": "3.9.0", + "resolved": "https://registry.npmjs.org/core-js-compat/-/core-js-compat-3.9.0.tgz", + "integrity": "sha512-YK6fwFjCOKWwGnjFUR3c544YsnA/7DoLL0ysncuOJ4pwbriAtOpvM2bygdlcXbvQCQZ7bBU9CL4t7tGl7ETRpQ==", + "dependencies": { + "browserslist": "^4.16.3", + "semver": "7.0.0" + } + }, + "node_modules/gatsby-legacy-polyfills/node_modules/semver": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.0.0.tgz", + "integrity": "sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/gatsby-link": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-link/-/gatsby-link-4.14.0.tgz", + "integrity": "sha512-zaMhjalc5RL4knxcJJyBee3wfIjZzcQGTdytHjS1HwOLV/cewIQZFCKIeScgg/623bRFCqQAqB3dzcwAbxhq3A==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@types/reach__router": "^1.3.10", + "gatsby-page-utils": "^2.14.0", + "prop-types": "^15.8.1" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "@gatsbyjs/reach-router": "^1.3.5", + "react": "^16.9.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.9.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/gatsby-page-utils": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-page-utils/-/gatsby-page-utils-2.14.0.tgz", + "integrity": "sha512-7o2NEJIMFPmcwwynED9osHsL4byIRNolYUUijxzNgnVQfvRnmuSZZABoC3nF0uInVvAQcE496vGGj284SfyOLw==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "bluebird": "^3.7.2", + "chokidar": "^3.5.2", + "fs-exists-cached": "^1.0.0", + "gatsby-core-utils": "^3.14.0", + "glob": "^7.2.0", + "lodash": "^4.17.21", + "micromatch": "^4.0.5" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby-parcel-config": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/gatsby-parcel-config/-/gatsby-parcel-config-0.5.0.tgz", + "integrity": "sha512-Ff4MD1y9+tYLlzQ377TiW79L0+PQxTc8FKm+l6bYDs9LDmPf4I+tshIIJnQEJE7aLuR66Ow9qSdluZj2Df2msA==", + "dependencies": { + "@gatsbyjs/parcel-namer-relative-to-cwd": "0.0.2", + "@parcel/bundler-default": "^2.3.2", + "@parcel/compressor-raw": "^2.3.2", + "@parcel/namer-default": "^2.3.2", + "@parcel/optimizer-terser": "^2.3.2", + "@parcel/packager-js": "^2.3.2", + "@parcel/packager-raw": "^2.3.2", + "@parcel/reporter-dev-server": "^2.3.2", + "@parcel/resolver-default": "^2.3.2", + "@parcel/runtime-browser-hmr": "^2.3.2", + "@parcel/runtime-js": "^2.3.2", + "@parcel/runtime-react-refresh": "^2.3.2", + "@parcel/runtime-service-worker": "^2.3.2", + "@parcel/transformer-js": "^2.3.2", + "@parcel/transformer-json": "^2.3.2", + "@parcel/transformer-raw": "^2.3.2", + "@parcel/transformer-react-refresh-wrap": "^2.3.2" + }, + "engines": { + "parcel": "2.x" + }, + "peerDependencies": { + "@parcel/core": "^2.3.1" + } + }, + "node_modules/gatsby-plugin-gatsby-cloud": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-gatsby-cloud/-/gatsby-plugin-gatsby-cloud-4.14.0.tgz", + "integrity": "sha512-9xG+9L5I/Dj2PgpXeTdwFu4FhEjbpAHJaQJuxi9Wz2h4O3ix02nkXmvcDrX3JC/peOQ4zFfHh/mTy2jHl2pzow==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "date-fns": "^2.28.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-telemetry": "^3.14.0", + "js-cookie": "^3.0.1", + "kebab-hash": "^0.1.2", + "lodash": "^4.17.21", + "webpack-assets-manifest": "^5.1.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next", + "webpack": "*" + } + }, + "node_modules/gatsby-plugin-image": { + "version": "2.14.1", + "resolved": "https://registry.npmjs.org/gatsby-plugin-image/-/gatsby-plugin-image-2.14.1.tgz", + "integrity": "sha512-9ez7/MqjZLwvFqY37miFO3IAcqF3rkhUMyrwct+JMk2c1AcXgGyG1acgKGkJU6maiQoIqSKtyzjoQD8+o3f0Hg==", + "dependencies": { + "@babel/code-frame": "^7.14.0", + "@babel/parser": "^7.15.5", + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "babel-jsx-utils": "^1.1.0", + "babel-plugin-remove-graphql-queries": "^4.14.0", + "camelcase": "^5.3.1", + "chokidar": "^3.5.2", + "common-tags": "^1.8.2", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "objectFitPolyfill": "^2.3.5", + "prop-types": "^15.8.1" + }, + "peerDependencies": { + "@babel/core": "^7.12.3", + "gatsby": "^4.0.0-next", + "gatsby-plugin-sharp": "^4.0.0-next", + "gatsby-source-filesystem": "^4.0.0-next", + "react": "^16.9.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.9.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/gatsby-plugin-manifest": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-manifest/-/gatsby-plugin-manifest-4.14.0.tgz", + "integrity": "sha512-vwN5ZasR6/I9Xd1Ar3+UhMaYN6EU7PJhJKY6aQMAtG1Qxva5lDjmrWNzUlm8NHL/XmB4VSS+A4TZUZHyoygZ7Q==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0", + "gatsby-plugin-utils": "^3.8.0", + "semver": "^7.3.7", + "sharp": "^0.30.3" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-plugin-mantine": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/gatsby-plugin-mantine/-/gatsby-plugin-mantine-4.0.0.tgz", + "integrity": "sha512-7Cc0f01H7DEsuLhe+0Qst0oKlZB/jhxRCL4WMW7m+fSc3s57fBrL44ObhFlWCWQtLJkSsOx9Fl4Se4FN+gwySA==", + "dependencies": { + "@mantine/ssr": ">=3.6.14" + }, + "peerDependencies": { + "gatsby": "*", + "react": "*", + "react-dom": "*" + } + }, + "node_modules/gatsby-plugin-offline": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-offline/-/gatsby-plugin-offline-5.14.0.tgz", + "integrity": "sha512-7ClwsPdX3ZEaR6BAKvXVRn/lh5X08i7xugXpI+qDXWfCFOmuWh3iy4fl1ESPxWRhem99bvv5ocqhGQ0Q1tAbHQ==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "cheerio": "^1.0.0-rc.10", + "gatsby-core-utils": "^3.14.0", + "glob": "^7.2.0", + "idb-keyval": "^3.2.0", + "lodash": "^4.17.21", + "workbox-build": "^4.3.1" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next", + "react": "^16.9.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.9.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/gatsby-plugin-page-creator": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-page-creator/-/gatsby-plugin-page-creator-4.14.0.tgz", + "integrity": "sha512-bDKnnXJ0H7fUNgEqvUWhPW6N3HkYmQbUQunCYTNl0tON9rLm8Rb0PwdlpCTeg20DOORMhIJZ3Uk6o2KMTGVlAQ==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "@sindresorhus/slugify": "^1.1.2", + "chokidar": "^3.5.2", + "fs-exists-cached": "^1.0.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-page-utils": "^2.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-telemetry": "^3.14.0", + "globby": "^11.1.0", + "lodash": "^4.17.21" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-plugin-react-helmet": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-react-helmet/-/gatsby-plugin-react-helmet-5.14.0.tgz", + "integrity": "sha512-Loy0fUtVDhEuvv1nCpsyyUpAosGRUB3g5GzHvV3oqxcGgrpTgeXTPH64drXuRKcHyJbH1vgUOuPdlIEzXDbQMQ==", + "dependencies": { + "@babel/runtime": "^7.15.4" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next", + "react-helmet": "^5.1.3 || ^6.0.0" + } + }, + "node_modules/gatsby-plugin-sass": { + "version": "5.14.0", + "resolved": "https://registry.npmmirror.com/gatsby-plugin-sass/-/gatsby-plugin-sass-5.14.0.tgz", + "integrity": "sha512-gRts8Tfb1AyNey+xPTRVFlc+agLM7DDZlh4V665RsWja9vF/bQlWwvX1DjmHgYkeVobh0dMyd9QV4TyDv9Uurw==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "resolve-url-loader": "^3.1.4", + "sass-loader": "^10.1.1" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next", + "sass": "^1.30.0" + } + }, + "node_modules/gatsby-plugin-sharp": { + "version": "4.14.1", + "resolved": "https://registry.npmjs.org/gatsby-plugin-sharp/-/gatsby-plugin-sharp-4.14.1.tgz", + "integrity": "sha512-izYl9XcnWdAY+rDtZtnKfzZ/djBioGSDg/k84cgLVbTzLB29jgqipw3PkjV7IlvmJmlw2v78YCHO5mk3bfcitg==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "async": "^3.2.3", + "bluebird": "^3.7.2", + "debug": "^4.3.4", + "filenamify": "^4.3.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-telemetry": "^3.14.0", + "got": "^11.8.3", + "lodash": "^4.17.21", + "mini-svg-data-uri": "^1.4.4", + "probe-image-size": "^7.2.3", + "progress": "^2.0.3", + "semver": "^7.3.7", + "sharp": "^0.30.3", + "svgo": "1.3.2", + "uuid": "3.4.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/async": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/async/-/async-3.2.3.tgz", + "integrity": "sha512-spZRyzKL5l5BZQrr/6m/SqFdBN0q3OCI0f9rjfBzCMBIP4p75P620rR3gTmaksNOhmzgdxcaxdNfMy6anrbM0g==" + }, + "node_modules/gatsby-plugin-sharp/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/css-select": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/css-select/-/css-select-2.1.0.tgz", + "integrity": "sha512-Dqk7LQKpwLoH3VovzZnkzegqNSuAziQyNZUcrdDM401iY+R5NkGBXGmtO05/yaXQziALuPogeG0b7UAgjnTJTQ==", + "dependencies": { + "boolbase": "^1.0.0", + "css-what": "^3.2.1", + "domutils": "^1.7.0", + "nth-check": "^1.0.2" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/css-tree": { + "version": "1.0.0-alpha.37", + "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.0.0-alpha.37.tgz", + "integrity": "sha512-DMxWJg0rnz7UgxKT0Q1HU/L9BeJI0M6ksor0OgqOnF+aRCDWg/N2641HmVyU9KVIu0OVVWOb2IpC9A+BJRnejg==", + "dependencies": { + "mdn-data": "2.0.4", + "source-map": "^0.6.1" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/css-what": { + "version": "3.4.2", + "resolved": "https://registry.npmjs.org/css-what/-/css-what-3.4.2.tgz", + "integrity": "sha512-ACUm3L0/jiZTqfzRM3Hi9Q8eZqd6IK37mMWPLz9PJxkLWllYeRf+EHUSHYEtFop2Eqytaq1FizFVh7XfBnXCDQ==", + "engines": { + "node": ">= 6" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/dom-serializer": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.2.2.tgz", + "integrity": "sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g==", + "dependencies": { + "domelementtype": "^2.0.1", + "entities": "^2.0.0" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/dom-serializer/node_modules/domelementtype": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-2.3.0.tgz", + "integrity": "sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw==" + }, + "node_modules/gatsby-plugin-sharp/node_modules/domelementtype": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", + "integrity": "sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w==" + }, + "node_modules/gatsby-plugin-sharp/node_modules/domutils": { + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.7.0.tgz", + "integrity": "sha512-Lgd2XcJ/NjEw+7tFvfKxOzCYKZsdct5lczQ2ZaQY8Djz7pfAD3Gbp8ySJWtreII/vDlMVmxwa6pHmdxIYgttDg==", + "dependencies": { + "dom-serializer": "0", + "domelementtype": "1" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/mdn-data": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-2.0.4.tgz", + "integrity": "sha512-iV3XNKw06j5Q7mi6h+9vbx23Tv7JkjEVgKHW4pimwyDGWm0OIQntJJ+u1C6mg6mK1EaTv42XQ7w76yuzH7M2cA==" + }, + "node_modules/gatsby-plugin-sharp/node_modules/nth-check": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/nth-check/-/nth-check-1.0.2.tgz", + "integrity": "sha512-WeBOdju8SnzPN5vTUJYxYUxLeXpCaVP5i5e0LF8fg7WORF2Wd7wFX/pk0tYZk7s8T+J7VLy0Da6J1+wCT0AtHg==", + "dependencies": { + "boolbase": "~1.0.0" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/svgo": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/svgo/-/svgo-1.3.2.tgz", + "integrity": "sha512-yhy/sQYxR5BkC98CY7o31VGsg014AKLEPxdfhora76l36hD9Rdy5NZA/Ocn6yayNPgSamYdtX2rFJdcv07AYVw==", + "deprecated": "This SVGO version is no longer supported. Upgrade to v2.x.x.", + "dependencies": { + "chalk": "^2.4.1", + "coa": "^2.0.2", + "css-select": "^2.0.0", + "css-select-base-adapter": "^0.1.1", + "css-tree": "1.0.0-alpha.37", + "csso": "^4.0.2", + "js-yaml": "^3.13.1", + "mkdirp": "~0.5.1", + "object.values": "^1.1.0", + "sax": "~1.2.4", + "stable": "^0.1.8", + "unquote": "~1.1.1", + "util.promisify": "~1.0.0" + }, + "bin": { + "svgo": "bin/svgo" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/gatsby-plugin-sharp/node_modules/uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==", + "deprecated": "Please upgrade to version 7 or higher. Older versions may use Math.random() in certain circumstances, which is known to be problematic. See https://v8.dev/blog/math-random for details.", + "bin": { + "uuid": "bin/uuid" + } + }, + "node_modules/gatsby-plugin-typescript": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-typescript/-/gatsby-plugin-typescript-4.14.0.tgz", + "integrity": "sha512-iAeC1dnpj99hjnRpD4FetXaJ9b321AuIf0q9vAw4G9FvddG0pxDtg3X9roUV8cmJ+VaLNsLr0DYc4fvOfrFGUQ==", + "dependencies": { + "@babel/core": "^7.15.5", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.14.5", + "@babel/plugin-proposal-numeric-separator": "^7.14.5", + "@babel/plugin-proposal-optional-chaining": "^7.14.5", + "@babel/preset-typescript": "^7.15.0", + "@babel/runtime": "^7.15.4", + "babel-plugin-remove-graphql-queries": "^4.14.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-plugin-utils": { + "version": "3.8.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-utils/-/gatsby-plugin-utils-3.8.0.tgz", + "integrity": "sha512-dLFk+4E2BJrSuPz5/cLUyw4/dDbyMtruLww2XnFk34DVxg16FHIBYcY7p5IbfmDiBmMtlgJFqxBHj1zt8l6syw==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-sharp": "^0.8.0", + "graphql-compose": "^9.0.7", + "import-from": "^4.0.0", + "joi": "^17.4.2", + "mime": "^3.0.0", + "mini-svg-data-uri": "^1.4.4", + "svgo": "^2.8.0" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-plugin-utils/node_modules/mime": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-3.0.0.tgz", + "integrity": "sha512-jSCU7/VB1loIWBZe14aEYHU/+1UMEHoaO7qxCOVJOw9GgH72VAWppxNcjU+x9a2k3GSIBXNKxXQFqRvvZ7vr3A==", + "bin": { + "mime": "cli.js" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/gatsby-react-router-scroll": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-react-router-scroll/-/gatsby-react-router-scroll-5.14.0.tgz", + "integrity": "sha512-jyqAmmo2UK6v/qRfx8bqlRkjiSYtJRUWNb4nx3bpEIvMlN/vGdJtJ60LsGkRJ5g6U6MybfVX7kUFjgjZdgtqHA==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "prop-types": "^15.8.1" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "@gatsbyjs/reach-router": "^1.3.5", + "react": "^16.9.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.9.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/gatsby-sharp": { + "version": "0.8.0", + "resolved": "https://registry.npmjs.org/gatsby-sharp/-/gatsby-sharp-0.8.0.tgz", + "integrity": "sha512-As590vHGlCiN9iCWneJo/pJYZjkWykjaFvoKAwPcv6Twn3+6l7ExKOBe9v/WmigALU23dI6vWP0JYvXmmvNYBg==", + "dependencies": { + "@types/sharp": "^0.30.0", + "sharp": "^0.30.3" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby-source-filesystem": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-source-filesystem/-/gatsby-source-filesystem-4.14.0.tgz", + "integrity": "sha512-r6QTNS0Y8DAWN/cHtA3P/cRXE41TAa0QuOeK+6lnoxAbDmgGt2/EvfXy0wIDvKJTYLQQL0WgY5/pxbvsUqTS5g==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "chokidar": "^3.5.2", + "file-type": "^16.5.3", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "got": "^9.6.0", + "md5-file": "^5.0.0", + "mime": "^2.5.2", + "pretty-bytes": "^5.4.1", + "progress": "^2.0.3", + "valid-url": "^1.0.9", + "xstate": "^4.26.1" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/@sindresorhus/is": { + "version": "0.14.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-0.14.0.tgz", + "integrity": "sha512-9NET910DNaIPngYnLLPeg+Ogzqsi9uM4mSboU5y6p8S5DzMTVEsJZrawi+BoDNUVBa2DhJqQYUFvMDfgU062LQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/@szmarczak/http-timer": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-1.1.2.tgz", + "integrity": "sha512-XIB2XbzHTN6ieIjfIMV9hlVcfPU26s2vafYWQcZHWXHOxiaRZYEDKEwdl129Zyg50+foYV2jCgtrqSA6qNuNSA==", + "dependencies": { + "defer-to-connect": "^1.0.1" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/cacheable-request": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-6.1.0.tgz", + "integrity": "sha512-Oj3cAGPCqOZX7Rz64Uny2GYAZNliQSqfbePrgAQ1wKAihYmCUnraBtJtKcGR4xz7wF+LoJC+ssFZvv5BgF9Igg==", + "dependencies": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^3.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^4.1.0", + "responselike": "^1.0.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/cacheable-request/node_modules/get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/cacheable-request/node_modules/lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/decompress-response": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-3.3.0.tgz", + "integrity": "sha1-gKTdMjdIOEv6JICDYirt7Jgq3/M=", + "dependencies": { + "mimic-response": "^1.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/defer-to-connect": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-1.1.3.tgz", + "integrity": "sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ==" + }, + "node_modules/gatsby-source-filesystem/node_modules/get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/got": { + "version": "9.6.0", + "resolved": "https://registry.npmjs.org/got/-/got-9.6.0.tgz", + "integrity": "sha512-R7eWptXuGYxwijs0eV+v3o6+XH1IqVK8dJOEecQfTmkncw9AV4dcw/Dhxi8MdlqPthxxpZyizMzyg8RTmEsG+Q==", + "dependencies": { + "@sindresorhus/is": "^0.14.0", + "@szmarczak/http-timer": "^1.1.2", + "cacheable-request": "^6.0.0", + "decompress-response": "^3.3.0", + "duplexer3": "^0.1.4", + "get-stream": "^4.1.0", + "lowercase-keys": "^1.0.1", + "mimic-response": "^1.0.1", + "p-cancelable": "^1.0.0", + "to-readable-stream": "^1.0.0", + "url-parse-lax": "^3.0.0" + }, + "engines": { + "node": ">=8.6" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha1-Wx85evx11ne96Lz8Dkfh+aPZqJg=" + }, + "node_modules/gatsby-source-filesystem/node_modules/keyv": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-3.1.0.tgz", + "integrity": "sha512-9ykJ/46SN/9KPM/sichzQ7OvXyGDYKGTaDlKMGCAlg2UK8KRy4jb0d8sFc+0Tt0YYnThq8X2RZgCg74RPxgcVA==", + "dependencies": { + "json-buffer": "3.0.0" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/lowercase-keys": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-1.0.1.tgz", + "integrity": "sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/normalize-url": { + "version": "4.5.1", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-4.5.1.tgz", + "integrity": "sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/p-cancelable": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-1.1.0.tgz", + "integrity": "sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/gatsby-source-filesystem/node_modules/responselike": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-1.0.2.tgz", + "integrity": "sha1-kYcg7ztjHFZCvgaPFa3lpG9Loec=", + "dependencies": { + "lowercase-keys": "^1.0.0" + } + }, + "node_modules/gatsby-telemetry": { + "version": "3.14.0", + "resolved": "https://registry.npmjs.org/gatsby-telemetry/-/gatsby-telemetry-3.14.0.tgz", + "integrity": "sha512-QnlN3nvb+1gYsY6cIQKAuvkhx9uoOg71yuEYB0EFQdgcnyIbWlBVRHId8wOXoQHwRYFmatvxBmcKlVF8FCs61A==", + "hasInstallScript": true, + "dependencies": { + "@babel/code-frame": "^7.14.0", + "@babel/runtime": "^7.15.4", + "@turist/fetch": "^7.1.7", + "@turist/time": "^0.0.2", + "async-retry-ng": "^2.0.1", + "boxen": "^4.2.0", + "configstore": "^5.0.1", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "git-up": "^4.0.5", + "is-docker": "^2.2.1", + "lodash": "^4.17.21", + "node-fetch": "^2.6.7" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby-telemetry/node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-telemetry/node_modules/boxen": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/boxen/-/boxen-4.2.0.tgz", + "integrity": "sha512-eB4uT9RGzg2odpER62bBwSLvUeGC+WbRjjyyFhGsKnc8wp/m0+hQsMUvUe3H2V0D5vw0nBdO1hCJoZo5mKeuIQ==", + "dependencies": { + "ansi-align": "^3.0.0", + "camelcase": "^5.3.1", + "chalk": "^3.0.0", + "cli-boxes": "^2.2.0", + "string-width": "^4.1.0", + "term-size": "^2.1.0", + "type-fest": "^0.8.1", + "widest-line": "^3.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-telemetry/node_modules/chalk": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-3.0.0.tgz", + "integrity": "sha512-4D3B6Wf41KOYRFdszmDqMCGq5VV/uMAB273JILmO+3jAlh8X4qDtdtgCR3fxtbLEMzSx22QdhnDcJvu2u1fVwg==", + "dependencies": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-telemetry/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/gatsby-telemetry/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/gatsby-telemetry/node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-telemetry/node_modules/supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-telemetry/node_modules/type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/gatsby-transformer-remark": { + "version": "5.14.0", + "resolved": "https://registry.npmmirror.com/gatsby-transformer-remark/-/gatsby-transformer-remark-5.14.0.tgz", + "integrity": "sha512-QHGnNRx9M+YMZBiS3dqHCvmSLgpGAXo/WRl8hNtCyO3MfF1oZlbAYkUWtMwNuwIWAtGmf/SifxzzKhiWRgfNjA==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0", + "gray-matter": "^4.0.3", + "hast-util-raw": "^6.0.2", + "hast-util-to-html": "^7.1.3", + "lodash": "^4.17.21", + "mdast-util-to-hast": "^10.2.0", + "mdast-util-to-string": "^2.0.0", + "mdast-util-toc": "^5.1.0", + "remark": "^13.0.0", + "remark-footnotes": "^3.0.0", + "remark-gfm": "^1.0.0", + "remark-parse": "^9.0.0", + "remark-retext": "^4.0.0", + "remark-stringify": "^9.0.1", + "retext-english": "^3.0.4", + "sanitize-html": "^1.27.5", + "underscore.string": "^3.3.6", + "unified": "^9.2.2", + "unist-util-remove-position": "^3.0.0", + "unist-util-select": "^3.0.4", + "unist-util-visit": "^2.0.3" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next" + } + }, + "node_modules/gatsby-transformer-sharp": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-transformer-sharp/-/gatsby-transformer-sharp-4.14.0.tgz", + "integrity": "sha512-V8tpCavnmK05GK7LoSAUyd/KZBoZr2NK4IdyIbBdxOcXBqtsD/yrThOyLP00hEV03myTv/6nzSIBZdEL+AGG5g==", + "dependencies": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "bluebird": "^3.7.2", + "common-tags": "^1.8.2", + "fs-extra": "^10.1.0", + "probe-image-size": "^7.2.3", + "semver": "^7.3.7", + "sharp": "^0.30.3" + }, + "engines": { + "node": ">=14.15.0" + }, + "peerDependencies": { + "gatsby": "^4.0.0-next", + "gatsby-plugin-sharp": "^4.0.0-next" + } + }, + "node_modules/gatsby-worker": { + "version": "1.14.0", + "resolved": "https://registry.npmjs.org/gatsby-worker/-/gatsby-worker-1.14.0.tgz", + "integrity": "sha512-Zxa295xBIdgsjg0evBFetm8ctkzi7l1cbPJ8VR5440SV8Mun1d1iPJYl070UazNSYz7UK1lTf1B0ISJYUg31VQ==", + "dependencies": { + "@babel/core": "^7.15.5", + "@babel/runtime": "^7.15.4" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby/node_modules/gatsby-cli": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-cli/-/gatsby-cli-4.14.0.tgz", + "integrity": "sha512-jmLhrBNguZM8ldKpt1dmxbEZ4j/OtEdE1IpUCHoLGoCIZ7QGtleA2WHhn0R4GnoY0FVP7+pGWcmPpBXo63DBXA==", + "hasInstallScript": true, + "dependencies": { + "@babel/code-frame": "^7.14.0", + "@babel/core": "^7.15.5", + "@babel/generator": "^7.16.8", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/preset-typescript": "^7.16.7", + "@babel/runtime": "^7.15.4", + "@babel/template": "^7.16.7", + "@babel/types": "^7.16.8", + "@types/common-tags": "^1.8.1", + "better-opn": "^2.1.1", + "boxen": "^5.1.2", + "chalk": "^4.1.2", + "clipboardy": "^2.3.0", + "common-tags": "^1.8.2", + "configstore": "^5.0.1", + "convert-hrtime": "^3.0.0", + "create-gatsby": "^2.14.0", + "envinfo": "^7.8.1", + "execa": "^5.1.1", + "fs-exists-cached": "^1.0.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-telemetry": "^3.14.0", + "hosted-git-info": "^3.0.8", + "is-valid-path": "^0.1.1", + "joi": "^17.4.2", + "lodash": "^4.17.21", + "meant": "^1.0.3", + "node-fetch": "^2.6.6", + "opentracing": "^0.14.5", + "pretty-error": "^2.1.2", + "progress": "^2.0.3", + "prompts": "^2.4.2", + "redux": "4.1.2", + "resolve-cwd": "^3.0.0", + "semver": "^7.3.7", + "signal-exit": "^3.0.6", + "source-map": "0.7.3", + "stack-trace": "^0.0.10", + "strip-ansi": "^6.0.1", + "update-notifier": "^5.1.0", + "uuid": "3.4.0", + "yargs": "^15.4.1", + "yoga-layout-prebuilt": "^1.10.0", + "yurnalist": "^2.1.0" + }, + "bin": { + "gatsby": "cli.js" + }, + "engines": { + "node": ">=14.15.0" + } + }, + "node_modules/gatsby/node_modules/gatsby-cli/node_modules/uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==", + "deprecated": "Please upgrade to version 7 or higher. Older versions may use Math.random() in certain circumstances, which is known to be problematic. See https://v8.dev/blog/math-random for details.", + "bin": { + "uuid": "bin/uuid" + } + }, + "node_modules/gauge": { + "version": "2.7.4", + "resolved": "https://registry.npmjs.org/gauge/-/gauge-2.7.4.tgz", + "integrity": "sha1-LANAXHU4w51+s3sxcCLjJfsBi/c=", + "dependencies": { + "aproba": "^1.0.3", + "console-control-strings": "^1.0.0", + "has-unicode": "^2.0.0", + "object-assign": "^4.1.0", + "signal-exit": "^3.0.0", + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1", + "wide-align": "^1.1.0" + } + }, + "node_modules/gauge/node_modules/ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gauge/node_modules/is-fullwidth-code-point": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz", + "integrity": "sha1-754xOG8DGn8NZDr4L95QxFfvAMs=", + "dependencies": { + "number-is-nan": "^1.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gauge/node_modules/string-width": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-1.0.2.tgz", + "integrity": "sha1-EYvfW4zcUaKn5w0hHgfisLmxB9M=", + "dependencies": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gauge/node_modules/strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", + "dependencies": { + "ansi-regex": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/gaze": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/gaze/-/gaze-1.1.3.tgz", + "integrity": "sha512-BRdNm8hbWzFzWHERTrejLqwHDfS4GibPoq5wjTPIoJHoBtKGPg3xAFfxmM+9ztbXelxcf2hwQcaz1PtmFeue8g==", + "dependencies": { + "globule": "^1.0.0" + }, + "engines": { + "node": ">= 4.0.0" + } + }, + "node_modules/gensync": { + "version": "1.0.0-beta.2", + "resolved": "https://registry.npmjs.org/gensync/-/gensync-1.0.0-beta.2.tgz", + "integrity": "sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/get-caller-file": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", + "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", + "engines": { + "node": "6.* || 8.* || >= 10.*" + } + }, + "node_modules/get-intrinsic": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/get-intrinsic/-/get-intrinsic-1.1.1.tgz", + "integrity": "sha512-kWZrnVM42QCiEA2Ig1bG8zjoIMOgxWwYCEeNdwY6Tv/cOSeGpcoX4pXHfKUxNKVoArnrEr2e9srnAxxGIraS9Q==", + "dependencies": { + "function-bind": "^1.1.1", + "has": "^1.0.3", + "has-symbols": "^1.0.1" + } + }, + "node_modules/get-own-enumerable-property-symbols": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/get-own-enumerable-property-symbols/-/get-own-enumerable-property-symbols-3.0.2.tgz", + "integrity": "sha512-I0UBV/XOz1XkIJHEUDMZAbzCThU/H8DxmSfmdGcKPnVhu2VfFqr34jr9777IyaTYvxjedWhqVIilEDsCdP5G6g==" + }, + "node_modules/get-port": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/get-port/-/get-port-3.2.0.tgz", + "integrity": "sha1-3Xzn3hh8Bsi/NTeWrHHgmfCYDrw=", + "engines": { + "node": ">=4" + } + }, + "node_modules/get-stdin": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/get-stdin/-/get-stdin-4.0.1.tgz", + "integrity": "sha512-F5aQMywwJ2n85s4hJPTT9RPxGmubonuB10MNYo17/xph174n2MIR33HRguhzVag10O/npM7SPk73LMZNP+FaWw==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/get-symbol-description": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/get-symbol-description/-/get-symbol-description-1.0.0.tgz", + "integrity": "sha512-2EmdH1YvIQiZpltCNgkuiUnyukzxM/R6NDJX31Ke3BG1Nq5b0S2PhX59UKi9vZpPDQVdqn+1IcaAwnzTT5vCjw==", + "dependencies": { + "call-bind": "^1.0.2", + "get-intrinsic": "^1.1.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/getpass": { + "version": "0.1.7", + "resolved": "https://registry.npmmirror.com/getpass/-/getpass-0.1.7.tgz", + "integrity": "sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==", + "dependencies": { + "assert-plus": "^1.0.0" + } + }, + "node_modules/gifwrap": { + "version": "0.9.4", + "resolved": "https://registry.npmjs.org/gifwrap/-/gifwrap-0.9.4.tgz", + "integrity": "sha512-MDMwbhASQuVeD4JKd1fKgNgCRL3fGqMM4WaqpNhWO0JiMOAjbQdumbs4BbBZEy9/M00EHEjKN3HieVhCUlwjeQ==", + "dependencies": { + "image-q": "^4.0.0", + "omggif": "^1.0.10" + } + }, + "node_modules/git-up": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/git-up/-/git-up-4.0.5.tgz", + "integrity": "sha512-YUvVDg/vX3d0syBsk/CKUTib0srcQME0JyHkL5BaYdwLsiCslPWmDSi8PUMo9pXYjrryMcmsCoCgsTpSCJEQaA==", + "dependencies": { + "is-ssh": "^1.3.0", + "parse-url": "^6.0.0" + } + }, + "node_modules/github-from-package": { + "version": "0.0.0", + "resolved": "https://registry.npmjs.org/github-from-package/-/github-from-package-0.0.0.tgz", + "integrity": "sha1-l/tdlr/eiXMxPyDoKI75oWf6ZM4=" + }, + "node_modules/github-slugger": { + "version": "1.4.0", + "resolved": "https://registry.npmmirror.com/github-slugger/-/github-slugger-1.4.0.tgz", + "integrity": "sha512-w0dzqw/nt51xMVmlaV1+JRzN+oCa1KfcgGEWhxUG16wbdA+Xnt/yoFO8Z8x/V82ZcZ0wy6ln9QDup5avbhiDhQ==" + }, + "node_modules/glob": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", + "integrity": "sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q==", + "dependencies": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + }, + "engines": { + "node": "*" + } + }, + "node_modules/glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "dependencies": { + "is-glob": "^4.0.1" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/glob-to-regexp": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz", + "integrity": "sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw==" + }, + "node_modules/global": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/global/-/global-4.4.0.tgz", + "integrity": "sha512-wv/LAoHdRE3BeTGz53FAamhGlPLhlssK45usmGFThIi4XqnBmjKQ16u+RNbP7WvigRZDxUsM0J3gcQ5yicaL0w==", + "dependencies": { + "min-document": "^2.19.0", + "process": "^0.11.10" + } + }, + "node_modules/global-dirs": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/global-dirs/-/global-dirs-3.0.0.tgz", + "integrity": "sha512-v8ho2DS5RiCjftj1nD9NmnfaOzTdud7RRnVd9kFNOjqZbISlx5DQ+OrTkywgd0dIt7oFCvKetZSHoHcP3sDdiA==", + "dependencies": { + "ini": "2.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/global-modules": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-2.0.0.tgz", + "integrity": "sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A==", + "dependencies": { + "global-prefix": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/global-prefix": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-3.0.0.tgz", + "integrity": "sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg==", + "dependencies": { + "ini": "^1.3.5", + "kind-of": "^6.0.2", + "which": "^1.3.1" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/global-prefix/node_modules/ini": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/ini/-/ini-1.3.8.tgz", + "integrity": "sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==" + }, + "node_modules/globals": { + "version": "11.12.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-11.12.0.tgz", + "integrity": "sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==", + "engines": { + "node": ">=4" + } + }, + "node_modules/globby": { + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/globby/-/globby-11.1.0.tgz", + "integrity": "sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g==", + "dependencies": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.2.9", + "ignore": "^5.2.0", + "merge2": "^1.4.1", + "slash": "^3.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/globule": { + "version": "1.3.3", + "resolved": "https://registry.npmmirror.com/globule/-/globule-1.3.3.tgz", + "integrity": "sha512-mb1aYtDbIjTu4ShMB85m3UzjX9BVKe9WCzsnfMSZk+K5GpIbBOexgg4PPCt5eHDEG5/ZQAUX2Kct02zfiPLsKg==", + "dependencies": { + "glob": "~7.1.1", + "lodash": "~4.17.10", + "minimatch": "~3.0.2" + }, + "engines": { + "node": ">= 0.10" + } + }, + "node_modules/globule/node_modules/glob": { + "version": "7.1.7", + "resolved": "https://registry.npmmirror.com/glob/-/glob-7.1.7.tgz", + "integrity": "sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ==", + "dependencies": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + }, + "engines": { + "node": "*" + } + }, + "node_modules/globule/node_modules/minimatch": { + "version": "3.0.8", + "resolved": "https://registry.npmmirror.com/minimatch/-/minimatch-3.0.8.tgz", + "integrity": "sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q==", + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, + "node_modules/got": { + "version": "11.8.3", + "resolved": "https://registry.npmjs.org/got/-/got-11.8.3.tgz", + "integrity": "sha512-7gtQ5KiPh1RtGS9/Jbv1ofDpBFuq42gyfEib+ejaRBJuj/3tQFeR5+gw57e4ipaU8c/rCjvX6fkQz2lyDlGAOg==", + "dependencies": { + "@sindresorhus/is": "^4.0.0", + "@szmarczak/http-timer": "^4.0.5", + "@types/cacheable-request": "^6.0.1", + "@types/responselike": "^1.0.0", + "cacheable-lookup": "^5.0.3", + "cacheable-request": "^7.0.2", + "decompress-response": "^6.0.0", + "http2-wrapper": "^1.0.0-beta.5.2", + "lowercase-keys": "^2.0.0", + "p-cancelable": "^2.0.0", + "responselike": "^2.0.0" + }, + "engines": { + "node": ">=10.19.0" + } + }, + "node_modules/graceful-fs": { + "version": "4.2.10", + "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.10.tgz", + "integrity": "sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA==" + }, + "node_modules/graphql": { + "version": "15.8.0", + "resolved": "https://registry.npmjs.org/graphql/-/graphql-15.8.0.tgz", + "integrity": "sha512-5gghUc24tP9HRznNpV2+FIoq3xKkj5dTQqf4v0CpdPbFVwFkWoxOM+o+2OC9ZSvjEMTjfmG9QT+gcvggTwW1zw==", + "engines": { + "node": ">= 10.x" + } + }, + "node_modules/graphql-compose": { + "version": "9.0.8", + "resolved": "https://registry.npmjs.org/graphql-compose/-/graphql-compose-9.0.8.tgz", + "integrity": "sha512-I3zvygpVz5hOWk2cYL6yhbgfKbNWbiZFNXlWkv/55U+lX6Y3tL+SyY3zunw7QWrN/qtwG2DqZb13SHTv2MgdEQ==", + "dependencies": { + "graphql-type-json": "0.3.2" + }, + "peerDependencies": { + "graphql": "^14.2.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/graphql-config": { + "version": "3.4.1", + "resolved": "https://registry.npmjs.org/graphql-config/-/graphql-config-3.4.1.tgz", + "integrity": "sha512-g9WyK4JZl1Ko++FSyE5Ir2g66njfxGzrDDhBOwnkoWf/t3TnnZG6BBkWP+pkqVJ5pqMJGPKHNrbew8jRxStjhw==", + "dependencies": { + "@endemolshinegroup/cosmiconfig-typescript-loader": "3.0.2", + "@graphql-tools/graphql-file-loader": "^6.0.0", + "@graphql-tools/json-file-loader": "^6.0.0", + "@graphql-tools/load": "^6.0.0", + "@graphql-tools/merge": "6.0.0 - 6.2.14", + "@graphql-tools/url-loader": "^6.0.0", + "@graphql-tools/utils": "^7.0.0", + "cosmiconfig": "7.0.0", + "cosmiconfig-toml-loader": "1.0.0", + "minimatch": "3.0.4", + "string-env-interpolation": "1.0.1" + }, + "engines": { + "node": ">= 10.0.0" + }, + "peerDependencies": { + "graphql": "^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0" + } + }, + "node_modules/graphql-config/node_modules/@graphql-tools/load": { + "version": "6.2.8", + "resolved": "https://registry.npmjs.org/@graphql-tools/load/-/load-6.2.8.tgz", + "integrity": "sha512-JpbyXOXd8fJXdBh2ta0Q4w8ia6uK5FHzrTNmcvYBvflFuWly2LDTk2abbSl81zKkzswQMEd2UIYghXELRg8eTA==", + "dependencies": { + "@graphql-tools/merge": "^6.2.12", + "@graphql-tools/utils": "^7.5.0", + "globby": "11.0.3", + "import-from": "3.0.0", + "is-glob": "4.0.1", + "p-limit": "3.1.0", + "tslib": "~2.2.0", + "unixify": "1.0.0", + "valid-url": "1.0.9" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/graphql-config/node_modules/@graphql-tools/merge": { + "version": "6.2.14", + "resolved": "https://registry.npmjs.org/@graphql-tools/merge/-/merge-6.2.14.tgz", + "integrity": "sha512-RWT4Td0ROJai2eR66NHejgf8UwnXJqZxXgDWDI+7hua5vNA2OW8Mf9K1Wav1ZkjWnuRp4ztNtkZGie5ISw55ow==", + "dependencies": { + "@graphql-tools/schema": "^7.0.0", + "@graphql-tools/utils": "^7.7.0", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/graphql-config/node_modules/@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "dependencies": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/graphql-config/node_modules/@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "dependencies": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "peerDependencies": { + "graphql": "^14.0.0 || ^15.0.0" + } + }, + "node_modules/graphql-config/node_modules/cosmiconfig": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.0.tgz", + "integrity": "sha512-pondGvTuVYDk++upghXJabWzL6Kxu6f26ljFw64Swq9v6sQPUL3EUlVDV56diOjpCayKihL6hVe8exIACU4XcA==", + "dependencies": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.2.1", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.10.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/graphql-config/node_modules/globby": { + "version": "11.0.3", + "resolved": "https://registry.npmjs.org/globby/-/globby-11.0.3.tgz", + "integrity": "sha512-ffdmosjA807y7+lA1NM0jELARVmYul/715xiILEjo3hBLPTcirgQNnXECn5g3mtR8TOLCVbkfua1Hpen25/Xcg==", + "dependencies": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.1.1", + "ignore": "^5.1.4", + "merge2": "^1.3.0", + "slash": "^3.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/graphql-config/node_modules/import-from": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/import-from/-/import-from-3.0.0.tgz", + "integrity": "sha512-CiuXOFFSzkU5x/CR0+z7T91Iht4CXgfCxVOFRhh2Zyhg5wOpWvvDLQUsWl+gcN+QscYBjez8hDCt85O7RLDttQ==", + "dependencies": { + "resolve-from": "^5.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/graphql-config/node_modules/is-glob": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.1.tgz", + "integrity": "sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg==", + "dependencies": { + "is-extglob": "^2.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/graphql-config/node_modules/minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, + "node_modules/graphql-config/node_modules/p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dependencies": { + "yocto-queue": "^0.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/graphql-config/node_modules/tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "node_modules/graphql-config/node_modules/value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==", + "engines": { + "node": ">=12" + } + }, + "node_modules/graphql-playground-html": { + "version": "1.6.30", + "resolved": "https://registry.npmjs.org/graphql-playground-html/-/graphql-playground-html-1.6.30.tgz", + "integrity": "sha512-tpCujhsJMva4aqE8ULnF7/l3xw4sNRZcSHu+R00VV+W0mfp+Q20Plvcrp+5UXD+2yS6oyCXncA+zoQJQqhGCEw==", + "dependencies": { + "xss": "^1.0.6" + } + }, + "node_modules/graphql-playground-middleware-express": { + "version": "1.7.23", + "resolved": "https://registry.npmjs.org/graphql-playground-middleware-express/-/graphql-playground-middleware-express-1.7.23.tgz", + "integrity": "sha512-M/zbTyC1rkgiQjFSgmzAv6umMHOphYLNWZp6Ye5QrD77WfGOOoSqDsVmGUczc2pDkEPEzzGB/bvBO5rdzaTRgw==", + "dependencies": { + "graphql-playground-html": "^1.6.30" + }, + "peerDependencies": { + "express": "^4.16.2" + } + }, + "node_modules/graphql-tag": { + "version": "2.12.6", + "resolved": "https://registry.npmjs.org/graphql-tag/-/graphql-tag-2.12.6.tgz", + "integrity": "sha512-FdSNcu2QQcWnM2VNvSCCDCVS5PpPqpzgFT8+GXzqJuoDd0CBncxCY278u4mhRO7tMgo2JjgJA5aZ+nWSQ/Z+xg==", + "dependencies": { + "tslib": "^2.1.0" + }, + "engines": { + "node": ">=10" + }, + "peerDependencies": { + "graphql": "^0.9.0 || ^0.10.0 || ^0.11.0 || ^0.12.0 || ^0.13.0 || ^14.0.0 || ^15.0.0 || ^16.0.0" + } + }, + "node_modules/graphql-type-json": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/graphql-type-json/-/graphql-type-json-0.3.2.tgz", + "integrity": "sha512-J+vjof74oMlCWXSvt0DOf2APEdZOCdubEvGDUAlqH//VBYcOYsGgRW7Xzorr44LvkjiuvecWc8fChxuZZbChtg==", + "peerDependencies": { + "graphql": ">=0.8.0" + } + }, + "node_modules/graphql-ws": { + "version": "4.9.0", + "resolved": "https://registry.npmjs.org/graphql-ws/-/graphql-ws-4.9.0.tgz", + "integrity": "sha512-sHkK9+lUm20/BGawNEWNtVAeJzhZeBg21VmvmLoT5NdGVeZWv5PdIhkcayQIAgjSyyQ17WMKmbDijIPG2On+Ag==", + "engines": { + "node": ">=10" + }, + "peerDependencies": { + "graphql": ">=0.11 <=15" + } + }, + "node_modules/gray-matter": { + "version": "4.0.3", + "resolved": "https://registry.npmmirror.com/gray-matter/-/gray-matter-4.0.3.tgz", + "integrity": "sha512-5v6yZd4JK3eMI3FqqCouswVqwugaA9r4dNZB1wwcmrD02QkV5H0y7XBQW8QwQqEaZY1pM9aqORSORhJRdNK44Q==", + "dependencies": { + "js-yaml": "^3.13.1", + "kind-of": "^6.0.2", + "section-matter": "^1.0.0", + "strip-bom-string": "^1.0.0" + }, + "engines": { + "node": ">=6.0" + } + }, + "node_modules/gzip-size": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-6.0.0.tgz", + "integrity": "sha512-ax7ZYomf6jqPTQ4+XCpUGyXKHk5WweS+e05MBO4/y3WJ5RkmPXNKvX+bx1behVILVwr6JSQvZAku021CHPXG3Q==", + "dependencies": { + "duplexer": "^0.1.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/har-schema": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/har-schema/-/har-schema-2.0.0.tgz", + "integrity": "sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==", + "engines": { + "node": ">=4" + } + }, + "node_modules/har-validator": { + "version": "5.1.5", + "resolved": "https://registry.npmmirror.com/har-validator/-/har-validator-5.1.5.tgz", + "integrity": "sha512-nmT2T0lljbxdQZfspsno9hgrG3Uir6Ks5afism62poxqBM6sDnMEuPmzTq8XN0OEwqKLLdh1jQI3qyE66Nzb3w==", + "deprecated": "this library is no longer supported", + "dependencies": { + "ajv": "^6.12.3", + "har-schema": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/hard-rejection": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/hard-rejection/-/hard-rejection-2.1.0.tgz", + "integrity": "sha512-VIZB+ibDhx7ObhAe7OVtoEbuP4h/MuOTHJ+J8h/eBXotJYl0fBgR72xDFCKgIh22OJZIOVNxBMWuhAr10r8HdA==", + "engines": { + "node": ">=6" + } + }, + "node_modules/has": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz", + "integrity": "sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==", + "dependencies": { + "function-bind": "^1.1.1" + }, + "engines": { + "node": ">= 0.4.0" + } + }, + "node_modules/has-ansi": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/has-ansi/-/has-ansi-2.0.0.tgz", + "integrity": "sha512-C8vBJ8DwUCx19vhm7urhTuUsr4/IyP6l4VzNQDv+ryHQObW3TTTp9yB68WpYgRe2bbaGuZ/se74IqFeVnMnLZg==", + "dependencies": { + "ansi-regex": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/has-ansi/node_modules/ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha512-TIGnTpdo+E3+pCyAluZvtED5p5wCqLdezCyhPZzKPcxvFplEt4i+W7OONCKgeZFT3+y5NZZfOOS/Bdcanm1MYA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/has-bigints": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/has-bigints/-/has-bigints-1.0.2.tgz", + "integrity": "sha512-tSvCKtBr9lkF0Ex0aQiP9N+OpV4zi2r/Nee5VkRDbaqv35RLYMzbwQfFSZZH0kR+Rd6302UJZ2p/bJCEoR3VoQ==" + }, + "node_modules/has-cors": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/has-cors/-/has-cors-1.1.0.tgz", + "integrity": "sha1-XkdHk/fqmEPRu5nCPu9J/xJv/zk=" + }, + "node_modules/has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "engines": { + "node": ">=4" + } + }, + "node_modules/has-property-descriptors": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz", + "integrity": "sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ==", + "dependencies": { + "get-intrinsic": "^1.1.1" + } + }, + "node_modules/has-symbols": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.3.tgz", + "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==", + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/has-tostringtag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-tostringtag/-/has-tostringtag-1.0.0.tgz", + "integrity": "sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ==", + "dependencies": { + "has-symbols": "^1.0.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/has-unicode": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/has-unicode/-/has-unicode-2.0.1.tgz", + "integrity": "sha1-4Ob+aijPUROIVeCG0Wkedx3iqLk=" + }, + "node_modules/has-yarn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/has-yarn/-/has-yarn-2.1.0.tgz", + "integrity": "sha512-UqBRqi4ju7T+TqGNdqAO0PaSVGsDGJUBQvk9eUWNGRY1CFGDzYhLWoM7JQEemnlvVcv/YEmc2wNW8BC24EnUsw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/hasha": { + "version": "5.2.2", + "resolved": "https://registry.npmjs.org/hasha/-/hasha-5.2.2.tgz", + "integrity": "sha512-Hrp5vIK/xr5SkeN2onO32H0MgNZ0f17HRNH39WfL0SYUNOTZ5Lz1TJ8Pajo/87dYGEFlLMm7mIc/k/s6Bvz9HQ==", + "dependencies": { + "is-stream": "^2.0.0", + "type-fest": "^0.8.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/hasha/node_modules/type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/hast-to-hyperscript": { + "version": "9.0.1", + "resolved": "https://registry.npmmirror.com/hast-to-hyperscript/-/hast-to-hyperscript-9.0.1.tgz", + "integrity": "sha512-zQgLKqF+O2F72S1aa4y2ivxzSlko3MAvxkwG8ehGmNiqd98BIN3JM1rAJPmplEyLmGLO2QZYJtIneOSZ2YbJuA==", + "dependencies": { + "@types/unist": "^2.0.3", + "comma-separated-tokens": "^1.0.0", + "property-information": "^5.3.0", + "space-separated-tokens": "^1.0.0", + "style-to-object": "^0.3.0", + "unist-util-is": "^4.0.0", + "web-namespaces": "^1.0.0" + } + }, + "node_modules/hast-util-from-parse5": { + "version": "6.0.1", + "resolved": "https://registry.npmmirror.com/hast-util-from-parse5/-/hast-util-from-parse5-6.0.1.tgz", + "integrity": "sha512-jeJUWiN5pSxW12Rh01smtVkZgZr33wBokLzKLwinYOUfSzm1Nl/c3GUGebDyOKjdsRgMvoVbV0VpAcpjF4NrJA==", + "dependencies": { + "@types/parse5": "^5.0.0", + "hastscript": "^6.0.0", + "property-information": "^5.0.0", + "vfile": "^4.0.0", + "vfile-location": "^3.2.0", + "web-namespaces": "^1.0.0" + } + }, + "node_modules/hast-util-is-element": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/hast-util-is-element/-/hast-util-is-element-1.1.0.tgz", + "integrity": "sha512-oUmNua0bFbdrD/ELDSSEadRVtWZOf3iF6Lbv81naqsIV99RnSCieTbWuWCY8BAeEfKJTKl0gRdokv+dELutHGQ==" + }, + "node_modules/hast-util-parse-selector": { + "version": "2.2.5", + "resolved": "https://registry.npmmirror.com/hast-util-parse-selector/-/hast-util-parse-selector-2.2.5.tgz", + "integrity": "sha512-7j6mrk/qqkSehsM92wQjdIgWM2/BW61u/53G6xmC8i1OmEdKLHbk419QKQUjz6LglWsfqoiHmyMRkP1BGjecNQ==" + }, + "node_modules/hast-util-raw": { + "version": "6.1.0", + "resolved": "https://registry.npmmirror.com/hast-util-raw/-/hast-util-raw-6.1.0.tgz", + "integrity": "sha512-5FoZLDHBpka20OlZZ4I/+RBw5piVQ8iI1doEvffQhx5CbCyTtP8UCq8Tw6NmTAMtXgsQxmhW7Ly8OdFre5/YMQ==", + "dependencies": { + "@types/hast": "^2.0.0", + "hast-util-from-parse5": "^6.0.0", + "hast-util-to-parse5": "^6.0.0", + "html-void-elements": "^1.0.0", + "parse5": "^6.0.0", + "unist-util-position": "^3.0.0", + "unist-util-visit": "^2.0.0", + "vfile": "^4.0.0", + "web-namespaces": "^1.0.0", + "xtend": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "node_modules/hast-util-to-html": { + "version": "7.1.3", + "resolved": "https://registry.npmmirror.com/hast-util-to-html/-/hast-util-to-html-7.1.3.tgz", + "integrity": "sha512-yk2+1p3EJTEE9ZEUkgHsUSVhIpCsL/bvT8E5GzmWc+N1Po5gBw+0F8bo7dpxXR0nu0bQVxVZGX2lBGF21CmeDw==", + "dependencies": { + "ccount": "^1.0.0", + "comma-separated-tokens": "^1.0.0", + "hast-util-is-element": "^1.0.0", + "hast-util-whitespace": "^1.0.0", + "html-void-elements": "^1.0.0", + "property-information": "^5.0.0", + "space-separated-tokens": "^1.0.0", + "stringify-entities": "^3.0.1", + "unist-util-is": "^4.0.0", + "xtend": "^4.0.0" + } + }, + "node_modules/hast-util-to-parse5": { + "version": "6.0.0", + "resolved": "https://registry.npmmirror.com/hast-util-to-parse5/-/hast-util-to-parse5-6.0.0.tgz", + "integrity": "sha512-Lu5m6Lgm/fWuz8eWnrKezHtVY83JeRGaNQ2kn9aJgqaxvVkFCZQBEhgodZUDUvoodgyROHDb3r5IxAEdl6suJQ==", + "dependencies": { + "hast-to-hyperscript": "^9.0.0", + "property-information": "^5.0.0", + "web-namespaces": "^1.0.0", + "xtend": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "node_modules/hast-util-whitespace": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/hast-util-whitespace/-/hast-util-whitespace-1.0.4.tgz", + "integrity": "sha512-I5GTdSfhYfAPNztx2xJRQpG8cuDSNt599/7YUn7Gx/WxNMsG+a835k97TDkFgk123cwjfwINaZknkKkphx/f2A==" + }, + "node_modules/hastscript": { + "version": "6.0.0", + "resolved": "https://registry.npmmirror.com/hastscript/-/hastscript-6.0.0.tgz", + "integrity": "sha512-nDM6bvd7lIqDUiYEiu5Sl/+6ReP0BMk/2f4U/Rooccxkj0P5nm+acM5PrGJ/t5I8qPGiqZSE6hVAwZEdZIvP4w==", + "dependencies": { + "@types/hast": "^2.0.0", + "comma-separated-tokens": "^1.0.0", + "hast-util-parse-selector": "^2.0.0", + "property-information": "^5.0.0", + "space-separated-tokens": "^1.0.0" + } + }, + "node_modules/header-case": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/header-case/-/header-case-2.0.4.tgz", + "integrity": "sha512-H/vuk5TEEVZwrR0lp2zed9OCo1uAILMlx0JEMgC26rzyJJ3N1v6XkwHHXJQdR2doSjcGPM6OKPYoJgf0plJ11Q==", + "dependencies": { + "capital-case": "^1.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/hoist-non-react-statics": { + "version": "3.3.2", + "resolved": "https://registry.npmmirror.com/hoist-non-react-statics/-/hoist-non-react-statics-3.3.2.tgz", + "integrity": "sha512-/gGivxi8JPKWNm/W0jSmzcMPpfpPLc3dY/6GxhX2hQ9iGj3aDfklV4ET7NjKpSinLpJ5vafa9iiGIEZg10SfBw==", + "dependencies": { + "react-is": "^16.7.0" + } + }, + "node_modules/hosted-git-info": { + "version": "3.0.8", + "resolved": "https://registry.npmjs.org/hosted-git-info/-/hosted-git-info-3.0.8.tgz", + "integrity": "sha512-aXpmwoOhRBrw6X3j0h5RloK4x1OzsxMPyxqIHyNfSe2pypkVTZFpEiRoSipPEPlMrh0HW/XsjkJ5WgnCirpNUw==", + "dependencies": { + "lru-cache": "^6.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/html-dom-parser": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/html-dom-parser/-/html-dom-parser-1.0.2.tgz", + "integrity": "sha512-Jq4oVkVSn+10ut3fyc2P/Fs1jqTo0l45cP6Q8d2ef/9jfkYwulO0QXmyLI0VUiZrXF4czpGgMEJRa52CQ6Fk8Q==", + "dependencies": { + "domhandler": "4.2.2", + "htmlparser2": "6.1.0" + } + }, + "node_modules/html-dom-parser/node_modules/domhandler": { + "version": "4.2.2", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-4.2.2.tgz", + "integrity": "sha512-PzE9aBMsdZO8TK4BnuJwH0QT41wgMbRzuZrHUcpYncEjmQazq8QEaBWgLG7ZyC/DAZKEgglpIA6j4Qn/HmxS3w==", + "dependencies": { + "domelementtype": "^2.2.0" + }, + "engines": { + "node": ">= 4" + } + }, + "node_modules/html-entities": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/html-entities/-/html-entities-2.3.3.tgz", + "integrity": "sha512-DV5Ln36z34NNTDgnz0EWGBLZENelNAtkiFA4kyNOG2tDI6Mz1uSWiq1wAKdyjnJwyDiDO7Fa2SO1CTxPXL8VxA==" + }, + "node_modules/html-react-parser": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/html-react-parser/-/html-react-parser-1.3.0.tgz", + "integrity": "sha512-lhpkOFH8pwqEjlNUYCWvjT43/JVCZO9MAZuCS6afT1/VP+bZcNxNUs4AUqiMzH0QPSDHwM/GFNXZNok1KTA4BQ==", + "dependencies": { + "domhandler": "4.2.2", + "html-dom-parser": "1.0.2", + "react-property": "2.0.0", + "style-to-js": "1.1.0" + }, + "peerDependencies": { + "react": "0.14 || 15 || 16 || 17" + } + }, + "node_modules/html-react-parser/node_modules/domhandler": { + "version": "4.2.2", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-4.2.2.tgz", + "integrity": "sha512-PzE9aBMsdZO8TK4BnuJwH0QT41wgMbRzuZrHUcpYncEjmQazq8QEaBWgLG7ZyC/DAZKEgglpIA6j4Qn/HmxS3w==", + "dependencies": { + "domelementtype": "^2.2.0" + }, + "engines": { + "node": ">= 4" + } + }, + "node_modules/html-tokenize": { + "version": "2.0.1", + "resolved": "https://registry.npmmirror.com/html-tokenize/-/html-tokenize-2.0.1.tgz", + "integrity": "sha512-QY6S+hZ0f5m1WT8WffYN+Hg+xm/w5I8XeUcAq/ZYP5wVC8xbKi4Whhru3FtrAebD5EhBW8rmFzkDI6eCAuFe2w==", + "dependencies": { + "buffer-from": "~0.1.1", + "inherits": "~2.0.1", + "minimist": "~1.2.5", + "readable-stream": "~1.0.27-1", + "through2": "~0.4.1" + }, + "bin": { + "html-tokenize": "bin/cmd.js" + } + }, + "node_modules/html-tokenize/node_modules/buffer-from": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/buffer-from/-/buffer-from-0.1.2.tgz", + "integrity": "sha512-RiWIenusJsmI2KcvqQABB83tLxCByE3upSP8QU3rJDMVFGPWLvPQJt/O1Su9moRWeH7d+Q2HYb68f6+v+tw2vg==" + }, + "node_modules/html-tokenize/node_modules/isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmmirror.com/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==" + }, + "node_modules/html-tokenize/node_modules/readable-stream": { + "version": "1.0.34", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.0.34.tgz", + "integrity": "sha512-ok1qVCJuRkNmvebYikljxJA/UEsKwLl2nI1OmaqAu4/UE+h0wKCHok4XkL/gvi39OacXvw59RJUOFUkDib2rHg==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "node_modules/html-tokenize/node_modules/string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==" + }, + "node_modules/html-void-elements": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/html-void-elements/-/html-void-elements-1.0.5.tgz", + "integrity": "sha512-uE/TxKuyNIcx44cIWnjr/rfIATDH7ZaOMmstu0CwhFG1Dunhlp4OC6/NMbhiwoq5BpW0ubi303qnEk/PZj614w==" + }, + "node_modules/htmlparser2": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/htmlparser2/-/htmlparser2-6.1.0.tgz", + "integrity": "sha512-gyyPk6rgonLFEDGoeRgQNaEUvdJ4ktTmmUh/h2t7s+M8oPpIPxgNACWa+6ESR57kXstwqPiCut0V8NRpcwgU7A==", + "dependencies": { + "domelementtype": "^2.0.1", + "domhandler": "^4.0.0", + "domutils": "^2.5.2", + "entities": "^2.0.0" + } + }, + "node_modules/http-cache-semantics": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/http-cache-semantics/-/http-cache-semantics-4.1.0.tgz", + "integrity": "sha512-carPklcUh7ROWRK7Cv27RPtdhYhUsela/ue5/jKzjegVvXDqM2ILE9Q2BGn9JZJh1g87cp56su/FgQSzcWS8cQ==" + }, + "node_modules/http-errors": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/http-errors/-/http-errors-2.0.0.tgz", + "integrity": "sha512-FtwrG/euBzaEjYeRqOgly7G0qviiXoJWnvEH2Z1plBdXgbyjv34pHTSb9zoeHMyDy33+DWy5Wt9Wo+TURtOYSQ==", + "dependencies": { + "depd": "2.0.0", + "inherits": "2.0.4", + "setprototypeof": "1.2.0", + "statuses": "2.0.1", + "toidentifier": "1.0.1" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/http-proxy": { + "version": "1.18.1", + "resolved": "https://registry.npmjs.org/http-proxy/-/http-proxy-1.18.1.tgz", + "integrity": "sha512-7mz/721AbnJwIVbnaSv1Cz3Am0ZLT/UBwkC92VlxhXv/k/BBQfM2fXElQNC27BVGr0uwUpplYPQM9LnaBMR5NQ==", + "dependencies": { + "eventemitter3": "^4.0.0", + "follow-redirects": "^1.0.0", + "requires-port": "^1.0.0" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/http-proxy/node_modules/eventemitter3": { + "version": "4.0.7", + "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-4.0.7.tgz", + "integrity": "sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw==" + }, + "node_modules/http-signature": { + "version": "1.2.0", + "resolved": "https://registry.npmmirror.com/http-signature/-/http-signature-1.2.0.tgz", + "integrity": "sha512-CAbnr6Rz4CYQkLYUtSNXxQPUH2gK8f3iWexVlsnMeD+GjlsQ0Xsy1cOX+mN3dtxYomRy21CiOzU8Uhw6OwncEQ==", + "dependencies": { + "assert-plus": "^1.0.0", + "jsprim": "^1.2.2", + "sshpk": "^1.7.0" + }, + "engines": { + "node": ">=0.8", + "npm": ">=1.3.7" + } + }, + "node_modules/http2-wrapper": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/http2-wrapper/-/http2-wrapper-1.0.3.tgz", + "integrity": "sha512-V+23sDMr12Wnz7iTcDeJr3O6AIxlnvT/bmaAAAP/Xda35C90p9599p0F1eHR/N1KILWSoWVAiOMFjBBXaXSMxg==", + "dependencies": { + "quick-lru": "^5.1.1", + "resolve-alpn": "^1.0.0" + }, + "engines": { + "node": ">=10.19.0" + } + }, + "node_modules/human-signals": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/human-signals/-/human-signals-2.1.0.tgz", + "integrity": "sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw==", + "engines": { + "node": ">=10.17.0" + } + }, + "node_modules/iconv-lite": { + "version": "0.4.24", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.24.tgz", + "integrity": "sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA==", + "dependencies": { + "safer-buffer": ">= 2.1.2 < 3" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/icss-utils": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/icss-utils/-/icss-utils-5.1.0.tgz", + "integrity": "sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA==", + "engines": { + "node": "^10 || ^12 || >= 14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/idb-keyval": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/idb-keyval/-/idb-keyval-3.2.0.tgz", + "integrity": "sha512-slx8Q6oywCCSfKgPgL0sEsXtPVnSbTLWpyiDcu6msHOyKOLari1TD1qocXVCft80umnkk3/Qqh3lwoFt8T/BPQ==" + }, + "node_modules/ieee754": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/ieee754/-/ieee754-1.2.1.tgz", + "integrity": "sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==" + }, + "node_modules/ignore": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.2.0.tgz", + "integrity": "sha512-CmxgYGiEPCLhfLnpPp1MoRmifwEIOgjcHXxOBjv7mY96c+eWScsOP9c112ZyLdWHi0FxHjI+4uVhKYp/gcdRmQ==", + "engines": { + "node": ">= 4" + } + }, + "node_modules/image-q": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/image-q/-/image-q-4.0.0.tgz", + "integrity": "sha512-PfJGVgIfKQJuq3s0tTDOKtztksibuUEbJQIYT3by6wctQo+Rdlh7ef4evJ5NCdxY4CfMbvFkocEwbl4BF8RlJw==", + "dependencies": { + "@types/node": "16.9.1" + } + }, + "node_modules/image-q/node_modules/@types/node": { + "version": "16.9.1", + "resolved": "https://registry.npmjs.org/@types/node/-/node-16.9.1.tgz", + "integrity": "sha512-QpLcX9ZSsq3YYUUnD3nFDY8H7wctAhQj/TFKL8Ya8v5fMm3CFXxo8zStsLAl780ltoYoo1WvKUVGBQK+1ifr7g==" + }, + "node_modules/immer": { + "version": "9.0.12", + "resolved": "https://registry.npmjs.org/immer/-/immer-9.0.12.tgz", + "integrity": "sha512-lk7UNmSbAukB5B6dh9fnh5D0bJTOFKxVg2cyJWTYrWRfhLrLMBquONcUs3aFq507hNoIZEDDh8lb8UtOizSMhA==" + }, + "node_modules/immutable": { + "version": "3.7.6", + "resolved": "https://registry.npmjs.org/immutable/-/immutable-3.7.6.tgz", + "integrity": "sha1-E7TTyxK++hVIKib+Gy665kAHHks=", + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/import-fresh": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/import-fresh/-/import-fresh-3.3.0.tgz", + "integrity": "sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw==", + "dependencies": { + "parent-module": "^1.0.0", + "resolve-from": "^4.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/import-fresh/node_modules/resolve-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-4.0.0.tgz", + "integrity": "sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==", + "engines": { + "node": ">=4" + } + }, + "node_modules/import-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/import-from/-/import-from-4.0.0.tgz", + "integrity": "sha512-P9J71vT5nLlDeV8FHs5nNxaLbrpfAV5cF5srvbZfpwpcJoM/xZR3hiv+q+SAnuSmuGbXMWud063iIMx/V/EWZQ==", + "engines": { + "node": ">=12.2" + } + }, + "node_modules/import-lazy": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/import-lazy/-/import-lazy-2.1.0.tgz", + "integrity": "sha1-BWmOPUXIjo1+nZLLBYTnfwlvPkM=", + "engines": { + "node": ">=4" + } + }, + "node_modules/imurmurhash": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", + "integrity": "sha1-khi5srkoojixPcT7a21XbyMUU+o=", + "engines": { + "node": ">=0.8.19" + } + }, + "node_modules/indent-string": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/indent-string/-/indent-string-4.0.0.tgz", + "integrity": "sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=", + "dependencies": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "node_modules/inherits": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + }, + "node_modules/ini": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ini/-/ini-2.0.0.tgz", + "integrity": "sha512-7PnF4oN3CvZF23ADhA5wRaYEQpJ8qygSkbtTXWBeXWXmEVRXK+1ITciHWwHhsjv1TmW0MgacIv6hEi5pX5NQdA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/inline-style-parser": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/inline-style-parser/-/inline-style-parser-0.1.1.tgz", + "integrity": "sha512-7NXolsK4CAS5+xvdj5OMMbI962hU/wvwoxk+LWR9Ek9bVtyuuYScDN6eS0rUm6TxApFpw7CX1o4uJzcd4AyD3Q==" + }, + "node_modules/inquirer": { + "version": "7.3.3", + "resolved": "https://registry.npmjs.org/inquirer/-/inquirer-7.3.3.tgz", + "integrity": "sha512-JG3eIAj5V9CwcGvuOmoo6LB9kbAYT8HXffUl6memuszlwDC/qvFAJw49XJ5NROSFNPxp3iQg1GqkFhaY/CR0IA==", + "dependencies": { + "ansi-escapes": "^4.2.1", + "chalk": "^4.1.0", + "cli-cursor": "^3.1.0", + "cli-width": "^3.0.0", + "external-editor": "^3.0.3", + "figures": "^3.0.0", + "lodash": "^4.17.19", + "mute-stream": "0.0.8", + "run-async": "^2.4.0", + "rxjs": "^6.6.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0", + "through": "^2.3.6" + }, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/inquirer/node_modules/ansi-escapes": { + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-4.3.2.tgz", + "integrity": "sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==", + "dependencies": { + "type-fest": "^0.21.3" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/inquirer/node_modules/type-fest": { + "version": "0.21.3", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz", + "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==", + "engines": { + "node": ">=10" + } + }, + "node_modules/internal-slot": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/internal-slot/-/internal-slot-1.0.3.tgz", + "integrity": "sha512-O0DB1JC/sPyZl7cIo78n5dR7eUSwwpYPiXRhTzNxZVAMUuB8vlnRFyLxdrVToks6XPLVnFfbzaVd5WLjhgg+vA==", + "dependencies": { + "get-intrinsic": "^1.1.0", + "has": "^1.0.3", + "side-channel": "^1.0.4" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/invariant": { + "version": "2.2.4", + "resolved": "https://registry.npmjs.org/invariant/-/invariant-2.2.4.tgz", + "integrity": "sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA==", + "dependencies": { + "loose-envify": "^1.0.0" + } + }, + "node_modules/ipaddr.js": { + "version": "1.9.1", + "resolved": "https://registry.npmjs.org/ipaddr.js/-/ipaddr.js-1.9.1.tgz", + "integrity": "sha512-0KI/607xoxSToH7GjN1FfSbLoU0+btTicjsQSWQlh/hZykN8KpmMf7uYwPW3R+akZ6R/w18ZlXSHBYXiYUPO3g==", + "engines": { + "node": ">= 0.10" + } + }, + "node_modules/is-absolute": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-absolute/-/is-absolute-1.0.0.tgz", + "integrity": "sha512-dOWoqflvcydARa360Gvv18DZ/gRuHKi2NU/wU5X1ZFzdYfH29nkiNZsF3mp4OJ3H4yo9Mx8A/uAGNzpzPN3yBA==", + "dependencies": { + "is-relative": "^1.0.0", + "is-windows": "^1.0.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-absolute-url": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/is-absolute-url/-/is-absolute-url-3.0.3.tgz", + "integrity": "sha512-opmNIX7uFnS96NtPmhWQgQx6/NYFgsUXYMllcfzwWKUMwfo8kku1TvE6hkNcH+Q1ts5cMVrsY7j0bxXQDciu9Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-alphabetical": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-alphabetical/-/is-alphabetical-1.0.4.tgz", + "integrity": "sha512-DwzsA04LQ10FHTZuL0/grVDk4rFoVH1pjAToYwBrHSxcrBIGQuXrQMtD5U1b0U2XVgKZCTLLP8u2Qxqhy3l2Vg==" + }, + "node_modules/is-alphanumerical": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-alphanumerical/-/is-alphanumerical-1.0.4.tgz", + "integrity": "sha512-UzoZUr+XfVz3t3v4KyGEniVL9BDRoQtY7tOyrRybkVNjDFWyo1yhXNGrrBTQxp3ib9BLAWs7k2YKBQsFRkZG9A==", + "dependencies": { + "is-alphabetical": "^1.0.0", + "is-decimal": "^1.0.0" + } + }, + "node_modules/is-arrayish": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz", + "integrity": "sha1-d8mYQFJ6qOyxqLppe4BkWnqSap0=" + }, + "node_modules/is-bigint": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/is-bigint/-/is-bigint-1.0.4.tgz", + "integrity": "sha512-zB9CruMamjym81i2JZ3UMn54PKGsQzsJeo6xvN3HJJ4CAsQNB6iRutp2To77OfCNuoxspsIhzaPoO1zyCEhFOg==", + "dependencies": { + "has-bigints": "^1.0.1" + } + }, + "node_modules/is-binary-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", + "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "dependencies": { + "binary-extensions": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-boolean-object": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/is-boolean-object/-/is-boolean-object-1.1.2.tgz", + "integrity": "sha512-gDYaKHJmnj4aWxyj6YHyXVpdQawtVLHU5cb+eztPGczf6cjuTdwve5ZIEfgXqH4e57An1D1AKf8CZ3kYrQRqYA==", + "dependencies": { + "call-bind": "^1.0.2", + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-buffer": { + "version": "2.0.5", + "resolved": "https://registry.npmmirror.com/is-buffer/-/is-buffer-2.0.5.tgz", + "integrity": "sha512-i2R6zNFDwgEHJyQUtJEk0XFi1i0dPFn/oqjK3/vPCcDeJvW5NQ83V8QbicfF1SupOaB0h8ntgBC2YiE7dfyctQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/is-callable": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/is-callable/-/is-callable-1.2.4.tgz", + "integrity": "sha512-nsuwtxZfMX67Oryl9LCQ+upnC0Z0BgpwntpS89m1H/TLF0zNfzfLMV/9Wa/6MZsj0acpEjAO0KF1xT6ZdLl95w==", + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-ci": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/is-ci/-/is-ci-2.0.0.tgz", + "integrity": "sha512-YfJT7rkpQB0updsdHLGWrvhBJfcfzNNawYDNIyQXJz0IViGf75O8EBPKSdvw2rF+LGCsX4FZ8tcr3b19LcZq4w==", + "dependencies": { + "ci-info": "^2.0.0" + }, + "bin": { + "is-ci": "bin.js" + } + }, + "node_modules/is-core-module": { + "version": "2.9.0", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.9.0.tgz", + "integrity": "sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A==", + "dependencies": { + "has": "^1.0.3" + } + }, + "node_modules/is-date-object": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/is-date-object/-/is-date-object-1.0.5.tgz", + "integrity": "sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ==", + "dependencies": { + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-decimal": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-decimal/-/is-decimal-1.0.4.tgz", + "integrity": "sha512-RGdriMmQQvZ2aqaQq3awNA6dCGtKpiDFcOzrTWrDAT2MiWrKQVPmxLGHl7Y2nNu6led0kEyoX0enY0qXYsv9zw==" + }, + "node_modules/is-docker": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/is-docker/-/is-docker-2.2.1.tgz", + "integrity": "sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ==", + "bin": { + "is-docker": "cli.js" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-extendable": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/is-extendable/-/is-extendable-0.1.1.tgz", + "integrity": "sha512-5BMULNob1vgFX6EjQw5izWDxrecWK9AM72rugNr0TFldMOi0fj6Jk+zeKIt0xGj4cEfQIJth4w3OKWOJ4f+AFw==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-fullwidth-code-point": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", + "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-function": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-function/-/is-function-1.0.2.tgz", + "integrity": "sha512-lw7DUp0aWXYg+CBCN+JKkcE0Q2RayZnSvnZBlwgxHBQhqt5pZNVy4Ri7H9GmmXkdu7LUthszM+Tor1u/2iBcpQ==" + }, + "node_modules/is-glob": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", + "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "dependencies": { + "is-extglob": "^2.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-hexadecimal": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-hexadecimal/-/is-hexadecimal-1.0.4.tgz", + "integrity": "sha512-gyPJuv83bHMpocVYoqof5VDiZveEoGoFL8m3BXNb2VW8Xs+rz9kqO8LOQ5DH6EsuvilT1ApazU0pyl+ytbPtlw==" + }, + "node_modules/is-installed-globally": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/is-installed-globally/-/is-installed-globally-0.4.0.tgz", + "integrity": "sha512-iwGqO3J21aaSkC7jWnHP/difazwS7SFeIqxv6wEtLU8Y5KlzFTjyqcSIT0d8s4+dDhKytsk9PJZ2BkS5eZwQRQ==", + "dependencies": { + "global-dirs": "^3.0.0", + "is-path-inside": "^3.0.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/is-invalid-path": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/is-invalid-path/-/is-invalid-path-0.1.0.tgz", + "integrity": "sha1-MHqFWzzxqTi0TqcNLGEQYFNxTzQ=", + "dependencies": { + "is-glob": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-invalid-path/node_modules/is-extglob": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-1.0.0.tgz", + "integrity": "sha1-rEaBd8SUNAWgkvyPKXYMb/xiBsA=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-invalid-path/node_modules/is-glob": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-2.0.1.tgz", + "integrity": "sha1-0Jb5JqPe1WAPP9/ZEZjLCIjC2GM=", + "dependencies": { + "is-extglob": "^1.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-lower-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-lower-case/-/is-lower-case-2.0.2.tgz", + "integrity": "sha512-bVcMJy4X5Og6VZfdOZstSexlEy20Sr0k/p/b2IlQJlfdKAQuMpiv5w2Ccxb8sKdRUNAG1PnHVHjFSdRDVS6NlQ==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/is-negative-zero": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-negative-zero/-/is-negative-zero-2.0.2.tgz", + "integrity": "sha512-dqJvarLawXsFbNDeJW7zAz8ItJ9cd28YufuuFzh0G8pNHjJMnY08Dv7sYX2uF5UpQOwieAeOExEYAWWfu7ZZUA==", + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-npm": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/is-npm/-/is-npm-5.0.0.tgz", + "integrity": "sha512-WW/rQLOazUq+ST/bCAVBp/2oMERWLsR7OrKyt052dNDk4DHcDE0/7QSXITlmi+VBcV13DfIbysG3tZJm5RfdBA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/is-number": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", + "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==", + "engines": { + "node": ">=0.12.0" + } + }, + "node_modules/is-number-object": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/is-number-object/-/is-number-object-1.0.7.tgz", + "integrity": "sha512-k1U0IRzLMo7ZlYIfzRu23Oh6MiIFasgpb9X76eqfFZAqwH44UI4KTBvBYIZ1dSL9ZzChTB9ShHfLkR4pdW5krQ==", + "dependencies": { + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-obj": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/is-obj/-/is-obj-2.0.0.tgz", + "integrity": "sha512-drqDG3cbczxxEJRoOXcOjtdp1J/lyp1mNn0xaznRs8+muBhgQcrnbspox5X5fOw0HnMnbfDzvnEMEtqDEJEo8w==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-path-inside": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-3.0.3.tgz", + "integrity": "sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-plain-obj": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-1.1.0.tgz", + "integrity": "sha512-yvkRyxmFKEOQ4pNXCmJG5AEQNlXJS5LaONXo5/cLdTZdWvsZ1ioJEonLGAosKlMWE8lwUy/bJzMjcw8az73+Fg==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-plain-object": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-2.0.4.tgz", + "integrity": "sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og==", + "dependencies": { + "isobject": "^3.0.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-promise": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/is-promise/-/is-promise-4.0.0.tgz", + "integrity": "sha512-hvpoI6korhJMnej285dSg6nu1+e6uxs7zG3BYAm5byqDsgJNWwxzM6z6iZiAgQR4TJ30JmBTOwqZUw3WlyH3AQ==" + }, + "node_modules/is-regex": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/is-regex/-/is-regex-1.1.4.tgz", + "integrity": "sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg==", + "dependencies": { + "call-bind": "^1.0.2", + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-regexp": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-regexp/-/is-regexp-1.0.0.tgz", + "integrity": "sha1-/S2INUXEa6xaYz57mgnof6LLUGk=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-relative": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-relative/-/is-relative-1.0.0.tgz", + "integrity": "sha512-Kw/ReK0iqwKeu0MITLFuj0jbPAmEiOsIwyIXvvbfa6QfmN9pkD1M+8pdk7Rl/dTKbH34/XBFMbgD4iMJhLQbGA==", + "dependencies": { + "is-unc-path": "^1.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-relative-url": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-relative-url/-/is-relative-url-3.0.0.tgz", + "integrity": "sha512-U1iSYRlY2GIMGuZx7gezlB5dp1Kheaym7zKzO1PV06mOihiWTXejLwm4poEJysPyXF+HtK/BEd0DVlcCh30pEA==", + "dependencies": { + "is-absolute-url": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-root": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-root/-/is-root-2.1.0.tgz", + "integrity": "sha512-AGOriNp96vNBd3HtU+RzFEc75FfR5ymiYv8E553I71SCeXBiMsVDUtdio1OEFvrPyLIQ9tVR5RxXIFe5PUFjMg==", + "engines": { + "node": ">=6" + } + }, + "node_modules/is-shared-array-buffer": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-shared-array-buffer/-/is-shared-array-buffer-1.0.2.tgz", + "integrity": "sha512-sqN2UDu1/0y6uvXyStCOzyhAjCSlHceFoMKJW8W9EU9cvic/QdsZ0kEU93HEy3IUEFZIiH/3w+AH/UQbPHNdhA==", + "dependencies": { + "call-bind": "^1.0.2" + } + }, + "node_modules/is-ssh": { + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/is-ssh/-/is-ssh-1.3.3.tgz", + "integrity": "sha512-NKzJmQzJfEEma3w5cJNcUMxoXfDjz0Zj0eyCalHn2E6VOwlzjZo0yuO2fcBSf8zhFuVCL/82/r5gRcoi6aEPVQ==", + "dependencies": { + "protocols": "^1.1.0" + } + }, + "node_modules/is-stream": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-2.0.1.tgz", + "integrity": "sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-string": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/is-string/-/is-string-1.0.7.tgz", + "integrity": "sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg==", + "dependencies": { + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-symbol": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/is-symbol/-/is-symbol-1.0.4.tgz", + "integrity": "sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg==", + "dependencies": { + "has-symbols": "^1.0.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/is-typedarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz", + "integrity": "sha1-5HnICFjfDBsR3dppQPlgEfzaSpo=" + }, + "node_modules/is-unc-path": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-unc-path/-/is-unc-path-1.0.0.tgz", + "integrity": "sha512-mrGpVd0fs7WWLfVsStvgF6iEJnbjDFZh9/emhRDcGWTduTfNHd9CHeUwH3gYIjdbwo4On6hunkztwOaAw0yllQ==", + "dependencies": { + "unc-path-regex": "^0.1.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-upper-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-upper-case/-/is-upper-case-2.0.2.tgz", + "integrity": "sha512-44pxmxAvnnAOwBg4tHPnkfvgjPwbc5QIsSstNU+YcJ1ovxVzCWpSGosPJOZh/a1tdl81fbgnLc9LLv+x2ywbPQ==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/is-valid-domain": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-valid-domain/-/is-valid-domain-0.1.6.tgz", + "integrity": "sha512-ZKtq737eFkZr71At8NxOFcP9O1K89gW3DkdrGMpp1upr/ueWjj+Weh4l9AI4rN0Gt8W2M1w7jrG2b/Yv83Ljpg==", + "dependencies": { + "punycode": "^2.1.1" + } + }, + "node_modules/is-valid-path": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/is-valid-path/-/is-valid-path-0.1.1.tgz", + "integrity": "sha1-EQ+f90w39mPh7HkV60UfLbk6yd8=", + "dependencies": { + "is-invalid-path": "^0.1.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-weakref": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-weakref/-/is-weakref-1.0.2.tgz", + "integrity": "sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ==", + "dependencies": { + "call-bind": "^1.0.2" + } + }, + "node_modules/is-windows": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-windows/-/is-windows-1.0.2.tgz", + "integrity": "sha512-eXK1UInq2bPmjyX6e3VHIzMLobc4J94i4AWn+Hpq3OU5KkrRC96OAcR3PRJ/pGu6m8TRnBHP9dkXQVsT/COVIA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-wsl": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/is-wsl/-/is-wsl-2.2.0.tgz", + "integrity": "sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww==", + "dependencies": { + "is-docker": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-yarn-global": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/is-yarn-global/-/is-yarn-global-0.3.0.tgz", + "integrity": "sha512-VjSeb/lHmkoyd8ryPVIKvOCn4D1koMqY+vqyjjUfc3xyKtP4dYOxM44sZrnqQSzSds3xyOrUTLTC9LVCVgLngw==" + }, + "node_modules/isarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-1.0.0.tgz", + "integrity": "sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE=" + }, + "node_modules/isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=" + }, + "node_modules/isobject": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", + "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/isomorphic-ws": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/isomorphic-ws/-/isomorphic-ws-4.0.1.tgz", + "integrity": "sha512-BhBvN2MBpWTaSHdWRb/bwdZJ1WaehQ2L1KngkCkfLUGF0mAWAT1sQUQacEmQ0jXkFw/czDXPNQSL5u2/Krsz1w==", + "peerDependencies": { + "ws": "*" + } + }, + "node_modules/isstream": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/isstream/-/isstream-0.1.2.tgz", + "integrity": "sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==" + }, + "node_modules/iterall": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/iterall/-/iterall-1.3.0.tgz", + "integrity": "sha512-QZ9qOMdF+QLHxy1QIpUHUU1D5pS2CG2P69LF6L6CPjPYA/XMOmKV3PZpawHoAjHNyB0swdVTRxdYT4tbBbxqwg==" + }, + "node_modules/jest-worker": { + "version": "26.6.2", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-26.6.2.tgz", + "integrity": "sha512-KWYVV1c4i+jbMpaBC+U++4Va0cp8OisU185o73T1vo99hqi7w8tSJfUXYswwqqrjzwxa6KpRK54WhPvwf5w6PQ==", + "dependencies": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^7.0.0" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/jest-worker/node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/jest-worker/node_modules/supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/jimp": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/jimp/-/jimp-0.16.1.tgz", + "integrity": "sha512-+EKVxbR36Td7Hfd23wKGIeEyHbxShZDX6L8uJkgVW3ESA9GiTEPK08tG1XI2r/0w5Ch0HyJF5kPqF9K7EmGjaw==", + "dependencies": { + "@babel/runtime": "^7.7.2", + "@jimp/custom": "^0.16.1", + "@jimp/plugins": "^0.16.1", + "@jimp/types": "^0.16.1", + "regenerator-runtime": "^0.13.3" + } + }, + "node_modules/joi": { + "version": "17.6.0", + "resolved": "https://registry.npmjs.org/joi/-/joi-17.6.0.tgz", + "integrity": "sha512-OX5dG6DTbcr/kbMFj0KGYxuew69HPcAE3K/sZpEV2nP6e/j/C0HV+HNiBPCASxdx5T7DMoa0s8UeHWMnb6n2zw==", + "dependencies": { + "@hapi/hoek": "^9.0.0", + "@hapi/topo": "^5.0.0", + "@sideway/address": "^4.1.3", + "@sideway/formula": "^3.0.0", + "@sideway/pinpoint": "^2.0.0" + } + }, + "node_modules/jpeg-js": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/jpeg-js/-/jpeg-js-0.4.2.tgz", + "integrity": "sha512-+az2gi/hvex7eLTMTlbRLOhH6P6WFdk2ITI8HJsaH2VqYO0I594zXSYEP+tf4FW+8Cy68ScDXoAsQdyQanv3sw==" + }, + "node_modules/js-base64": { + "version": "2.6.4", + "resolved": "https://registry.npmmirror.com/js-base64/-/js-base64-2.6.4.tgz", + "integrity": "sha512-pZe//GGmwJndub7ZghVHz7vjb2LgC1m8B07Au3eYqeqv9emhESByMXxaEgkUkEqJe87oBbSniGYoQNIBklc7IQ==" + }, + "node_modules/js-cookie": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/js-cookie/-/js-cookie-3.0.1.tgz", + "integrity": "sha512-+0rgsUXZu4ncpPxRL+lNEptWMOWl9etvPHc/koSRp6MPwpRYAhmk0dUG00J4bxVV3r9uUzfo24wW0knS07SKSw==", + "engines": { + "node": ">=12" + } + }, + "node_modules/js-tokens": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/js-tokens/-/js-tokens-4.0.0.tgz", + "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==" + }, + "node_modules/js-yaml": { + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.14.1.tgz", + "integrity": "sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g==", + "dependencies": { + "argparse": "^1.0.7", + "esprima": "^4.0.0" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/jsbn": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/jsbn/-/jsbn-0.1.1.tgz", + "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + }, + "node_modules/jsesc": { + "version": "2.5.2", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", + "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", + "bin": { + "jsesc": "bin/jsesc" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/json-buffer": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.1.tgz", + "integrity": "sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ==" + }, + "node_modules/json-loader": { + "version": "0.5.7", + "resolved": "https://registry.npmjs.org/json-loader/-/json-loader-0.5.7.tgz", + "integrity": "sha512-QLPs8Dj7lnf3e3QYS1zkCo+4ZwqOiF9d/nZnYozTISxXWCfNs9yuky5rJw4/W34s7POaNlbZmQGaB5NiXCbP4w==" + }, + "node_modules/json-parse-even-better-errors": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz", + "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==" + }, + "node_modules/json-schema": { + "version": "0.4.0", + "resolved": "https://registry.npmmirror.com/json-schema/-/json-schema-0.4.0.tgz", + "integrity": "sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==" + }, + "node_modules/json-schema-traverse": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz", + "integrity": "sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg==" + }, + "node_modules/json-source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/json-source-map/-/json-source-map-0.6.1.tgz", + "integrity": "sha512-1QoztHPsMQqhDq0hlXY5ZqcEdUzxQEIxgFkKl4WUp2pgShObl+9ovi4kRh2TfvAfxAoHOJ9vIMEqk3k4iex7tg==" + }, + "node_modules/json-stable-stringify-without-jsonify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz", + "integrity": "sha1-nbe1lJatPzz+8wp1FC0tkwrXJlE=" + }, + "node_modules/json-stringify-safe": { + "version": "5.0.1", + "resolved": "https://registry.npmmirror.com/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz", + "integrity": "sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==" + }, + "node_modules/json5": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-2.2.1.tgz", + "integrity": "sha512-1hqLFMSrGHRHxav9q9gNjJ5EXznIxGVO09xQRrwplcS8qs28pZ8s8hupZAmqDwZUmVZ2Qb2jnyPOWcDH8m8dlA==", + "bin": { + "json5": "lib/cli.js" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/jsonfile": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-6.1.0.tgz", + "integrity": "sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ==", + "dependencies": { + "universalify": "^2.0.0" + }, + "optionalDependencies": { + "graceful-fs": "^4.1.6" + } + }, + "node_modules/jsprim": { + "version": "1.4.2", + "resolved": "https://registry.npmmirror.com/jsprim/-/jsprim-1.4.2.tgz", + "integrity": "sha512-P2bSOMAc/ciLz6DzgjVlGJP9+BrJWu5UDGK70C2iweC5QBIeFf0ZXRvGjEj2uYgrY2MkAAhsSWHDWlFtEroZWw==", + "dependencies": { + "assert-plus": "1.0.0", + "extsprintf": "1.3.0", + "json-schema": "0.4.0", + "verror": "1.10.0" + }, + "engines": { + "node": ">=0.6.0" + } + }, + "node_modules/jsx-ast-utils": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/jsx-ast-utils/-/jsx-ast-utils-3.3.0.tgz", + "integrity": "sha512-XzO9luP6L0xkxwhIJMTJQpZo/eeN60K08jHdexfD569AGxeNug6UketeHXEhROoM8aR7EcUoOQmIhcJQjcuq8Q==", + "dependencies": { + "array-includes": "^3.1.4", + "object.assign": "^4.1.2" + }, + "engines": { + "node": ">=4.0" + } + }, + "node_modules/kebab-hash": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/kebab-hash/-/kebab-hash-0.1.2.tgz", + "integrity": "sha512-BTZpq3xgISmQmAVzkISy4eUutsUA7s4IEFlCwOBJjvSFOwyR7I+fza+tBc/rzYWK/NrmFHjfU1IhO3lu29Ib/w==", + "dependencies": { + "lodash.kebabcase": "^4.1.1" + } + }, + "node_modules/keyv": { + "version": "4.2.7", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-4.2.7.tgz", + "integrity": "sha512-HeOstD8SXvtWoQhMMBCelcUuZsiV7T7MwsADtOXT0KuwYP9nCxrSoMDeLXNDTLN3VFSuRp38JzoGbbTboq3QQw==", + "dependencies": { + "compress-brotli": "^1.3.8", + "json-buffer": "3.0.1" + } + }, + "node_modules/kind-of": { + "version": "6.0.3", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.3.tgz", + "integrity": "sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/kleur": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/kleur/-/kleur-3.0.3.tgz", + "integrity": "sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w==", + "engines": { + "node": ">=6" + } + }, + "node_modules/klona": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/klona/-/klona-2.0.5.tgz", + "integrity": "sha512-pJiBpiXMbt7dkzXe8Ghj/u4FfXOOa98fPW+bihOJ4SjnoijweJrNThJfd3ifXpXhREjpoF2mZVH1GfS9LV3kHQ==", + "engines": { + "node": ">= 8" + } + }, + "node_modules/language-subtag-registry": { + "version": "0.3.21", + "resolved": "https://registry.npmjs.org/language-subtag-registry/-/language-subtag-registry-0.3.21.tgz", + "integrity": "sha512-L0IqwlIXjilBVVYKFT37X9Ih11Um5NEl9cbJIuU/SwP/zEEAbBPOnEeeuxVMf45ydWQRDQN3Nqc96OgbH1K+Pg==" + }, + "node_modules/language-tags": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/language-tags/-/language-tags-1.0.5.tgz", + "integrity": "sha1-0yHbxNowuovzAk4ED6XBRmH5GTo=", + "dependencies": { + "language-subtag-registry": "~0.3.2" + } + }, + "node_modules/latest-version": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/latest-version/-/latest-version-5.1.0.tgz", + "integrity": "sha512-weT+r0kTkRQdCdYCNtkMwWXQTMEswKrFBkm4ckQOMVhhqhIMI1UT2hMj+1iigIhgSZm5gTmrRXBNoGUgaTY1xA==", + "dependencies": { + "package-json": "^6.3.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/levn": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/levn/-/levn-0.4.1.tgz", + "integrity": "sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ==", + "dependencies": { + "prelude-ls": "^1.2.1", + "type-check": "~0.4.0" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/lilconfig": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/lilconfig/-/lilconfig-2.0.5.tgz", + "integrity": "sha512-xaYmXZtTHPAw5m+xLN8ab9C+3a8YmV3asNSPOATITbtwrfbwaLJj8h66H1WMIpALCkqsIzK3h7oQ+PdX+LQ9Eg==", + "engines": { + "node": ">=10" + } + }, + "node_modules/lines-and-columns": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/lines-and-columns/-/lines-and-columns-1.2.4.tgz", + "integrity": "sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==" + }, + "node_modules/lmdb": { + "version": "2.2.6", + "resolved": "https://registry.npmjs.org/lmdb/-/lmdb-2.2.6.tgz", + "integrity": "sha512-UmQV0oZZcV3EN6rjcAjIiuWcc3MYZGWQ0GUYz46Ron5fuTa/dUow7WSQa6leFkvZIKVUdECBWVw96tckfEzUFQ==", + "hasInstallScript": true, + "dependencies": { + "msgpackr": "^1.5.4", + "nan": "^2.14.2", + "node-gyp-build": "^4.2.3", + "ordered-binary": "^1.2.4", + "weak-lru-cache": "^1.2.2" + } + }, + "node_modules/load-bmfont": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/load-bmfont/-/load-bmfont-1.4.1.tgz", + "integrity": "sha512-8UyQoYmdRDy81Brz6aLAUhfZLwr5zV0L3taTQ4hju7m6biuwiWiJXjPhBJxbUQJA8PrkvJ/7Enqmwk2sM14soA==", + "dependencies": { + "buffer-equal": "0.0.1", + "mime": "^1.3.4", + "parse-bmfont-ascii": "^1.0.3", + "parse-bmfont-binary": "^1.0.5", + "parse-bmfont-xml": "^1.1.4", + "phin": "^2.9.1", + "xhr": "^2.0.1", + "xtend": "^4.0.0" + } + }, + "node_modules/load-bmfont/node_modules/mime": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-1.6.0.tgz", + "integrity": "sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg==", + "bin": { + "mime": "cli.js" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/loader-runner": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/loader-runner/-/loader-runner-4.3.0.tgz", + "integrity": "sha512-3R/1M+yS3j5ou80Me59j7F9IMs4PXs3VqRrm0TU3AbKPxlmpoY1TNscJV/oGJXo8qCatFGTfDbY6W6ipGOYXfg==", + "engines": { + "node": ">=6.11.5" + } + }, + "node_modules/loader-utils": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.2.tgz", + "integrity": "sha512-TM57VeHptv569d/GKh6TAYdzKblwDNiumOdkFnejjD0XwTH87K90w3O7AiJRqdQoXygvi1VQTJTLGhJl7WqA7A==", + "dependencies": { + "big.js": "^5.2.2", + "emojis-list": "^3.0.0", + "json5": "^2.1.2" + }, + "engines": { + "node": ">=8.9.0" + } + }, + "node_modules/locate-path": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", + "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", + "dependencies": { + "p-locate": "^4.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/lock": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/lock/-/lock-1.1.0.tgz", + "integrity": "sha1-UxV0mdFlOxNspmRRBx/KYVcD+lU=" + }, + "node_modules/lockfile": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/lockfile/-/lockfile-1.0.4.tgz", + "integrity": "sha512-cvbTwETRfsFh4nHsL1eGWapU1XFi5Ot9E85sWAwia7Y7EgB7vfqcZhTKZ+l7hCGxSPoushMv5GKhT5PdLv03WA==", + "dependencies": { + "signal-exit": "^3.0.2" + } + }, + "node_modules/lodash": { + "version": "4.17.21", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz", + "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==" + }, + "node_modules/lodash._reinterpolate": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/lodash._reinterpolate/-/lodash._reinterpolate-3.0.0.tgz", + "integrity": "sha1-DM8tiRZq8Ds2Y8eWU4t1rG4RTZ0=" + }, + "node_modules/lodash.clonedeep": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.clonedeep/-/lodash.clonedeep-4.5.0.tgz", + "integrity": "sha1-4j8/nE+Pvd6HJSnBBxhXoIblzO8=" + }, + "node_modules/lodash.debounce": { + "version": "4.0.8", + "resolved": "https://registry.npmjs.org/lodash.debounce/-/lodash.debounce-4.0.8.tgz", + "integrity": "sha1-gteb/zCmfEAF/9XiUVMArZyk168=" + }, + "node_modules/lodash.deburr": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/lodash.deburr/-/lodash.deburr-4.1.0.tgz", + "integrity": "sha1-3bG7s+8HRYwBd7oH3hRCLLAz/5s=" + }, + "node_modules/lodash.every": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.every/-/lodash.every-4.6.0.tgz", + "integrity": "sha1-64mYS+vENkJ5uzrvu9HKGb+mxqc=" + }, + "node_modules/lodash.flatten": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.flatten/-/lodash.flatten-4.4.0.tgz", + "integrity": "sha1-8xwiIlqWMtK7+OSt2+8kCqdlph8=" + }, + "node_modules/lodash.flattendeep": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.flattendeep/-/lodash.flattendeep-4.4.0.tgz", + "integrity": "sha1-+wMJF/hqMTTlvJvsDWngAT3f7bI=" + }, + "node_modules/lodash.foreach": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.foreach/-/lodash.foreach-4.5.0.tgz", + "integrity": "sha1-Gmo16s5AEoDH8G3d7DUWWrJ+PlM=" + }, + "node_modules/lodash.get": { + "version": "4.4.2", + "resolved": "https://registry.npmjs.org/lodash.get/-/lodash.get-4.4.2.tgz", + "integrity": "sha1-LRd/ZS+jHpObRDjVNBSZ36OCXpk=" + }, + "node_modules/lodash.has": { + "version": "4.5.2", + "resolved": "https://registry.npmjs.org/lodash.has/-/lodash.has-4.5.2.tgz", + "integrity": "sha1-0Z9NwQlQWMzL4rDN9O4P5Ko3yGI=" + }, + "node_modules/lodash.kebabcase": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/lodash.kebabcase/-/lodash.kebabcase-4.1.1.tgz", + "integrity": "sha1-hImxyw0p/4gZXM7KRI/21swpXDY=" + }, + "node_modules/lodash.map": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.map/-/lodash.map-4.6.0.tgz", + "integrity": "sha1-dx7Hg540c9nEzeKLGTlMNWL09tM=" + }, + "node_modules/lodash.maxby": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.maxby/-/lodash.maxby-4.6.0.tgz", + "integrity": "sha1-CCJABo88eiJ6oAqDgOTzjPB4bj0=" + }, + "node_modules/lodash.memoize": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", + "integrity": "sha1-vMbEmkKihA7Zl/Mj6tpezRguC/4=" + }, + "node_modules/lodash.merge": { + "version": "4.6.2", + "resolved": "https://registry.npmjs.org/lodash.merge/-/lodash.merge-4.6.2.tgz", + "integrity": "sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ==" + }, + "node_modules/lodash.sortby": { + "version": "4.7.0", + "resolved": "https://registry.npmjs.org/lodash.sortby/-/lodash.sortby-4.7.0.tgz", + "integrity": "sha1-7dFMgk4sycHgsKG0K7UhBRakJDg=" + }, + "node_modules/lodash.template": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.template/-/lodash.template-4.5.0.tgz", + "integrity": "sha512-84vYFxIkmidUiFxidA/KjjH9pAycqW+h980j7Fuz5qxRtO9pgB7MDFTdys1N7A5mcucRiDyEq4fusljItR1T/A==", + "dependencies": { + "lodash._reinterpolate": "^3.0.0", + "lodash.templatesettings": "^4.0.0" + } + }, + "node_modules/lodash.templatesettings": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/lodash.templatesettings/-/lodash.templatesettings-4.2.0.tgz", + "integrity": "sha512-stgLz+i3Aa9mZgnjr/O+v9ruKZsPsndy7qPZOchbqk2cnTU1ZaldKK+v7m54WoKIyxiuMZTKT2H81F8BeAc3ZQ==", + "dependencies": { + "lodash._reinterpolate": "^3.0.0" + } + }, + "node_modules/lodash.truncate": { + "version": "4.4.2", + "resolved": "https://registry.npmjs.org/lodash.truncate/-/lodash.truncate-4.4.2.tgz", + "integrity": "sha1-WjUNoLERO4N+z//VgSy+WNbq4ZM=" + }, + "node_modules/lodash.uniq": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.uniq/-/lodash.uniq-4.5.0.tgz", + "integrity": "sha1-0CJTc662Uq3BvILklFM5qEJ1R3M=" + }, + "node_modules/lodash.without": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.without/-/lodash.without-4.4.0.tgz", + "integrity": "sha1-PNRXSgC2e643OpS3SHcmQFB7eqw=" + }, + "node_modules/longest-streak": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/longest-streak/-/longest-streak-2.0.4.tgz", + "integrity": "sha512-vM6rUVCVUJJt33bnmHiZEvr7wPT78ztX7rojL+LW51bHtLh6HTjx84LA5W4+oa6aKEJA7jJu5LR6vQRBpA5DVg==" + }, + "node_modules/loose-envify": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/loose-envify/-/loose-envify-1.4.0.tgz", + "integrity": "sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==", + "dependencies": { + "js-tokens": "^3.0.0 || ^4.0.0" + }, + "bin": { + "loose-envify": "cli.js" + } + }, + "node_modules/lower-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/lower-case/-/lower-case-2.0.2.tgz", + "integrity": "sha512-7fm3l3NAF9WfN6W3JOmf5drwpVqX78JtoGJ3A6W0a6ZnldM41w2fV5D490psKFTpMds8TJse/eHLFFsNHHjHgg==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/lower-case-first": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/lower-case-first/-/lower-case-first-2.0.2.tgz", + "integrity": "sha512-EVm/rR94FJTZi3zefZ82fLWab+GX14LJN4HrWBcuo6Evmsl9hEfnqxgcHCKb9q+mNf6EVdsjx/qucYFIIB84pg==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/lru-cache": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", + "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", + "dependencies": { + "yallist": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/lru-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/lru-queue/-/lru-queue-0.1.0.tgz", + "integrity": "sha1-Jzi9nw089PhEkMVzbEhpmsYyzaM=", + "dependencies": { + "es5-ext": "~0.10.2" + } + }, + "node_modules/make-dir": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-3.1.0.tgz", + "integrity": "sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw==", + "dependencies": { + "semver": "^6.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/make-dir/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/make-error": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/make-error/-/make-error-1.3.6.tgz", + "integrity": "sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw==" + }, + "node_modules/map-age-cleaner": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/map-age-cleaner/-/map-age-cleaner-0.1.3.tgz", + "integrity": "sha512-bJzx6nMoP6PDLPBFmg7+xRKeFZvFboMrGlxmNj9ClvX53KrmvM5bXFXEWjbz4cz1AFn+jWJ9z/DJSz7hrs0w3w==", + "dependencies": { + "p-defer": "^1.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/map-age-cleaner/node_modules/p-defer": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-defer/-/p-defer-1.0.0.tgz", + "integrity": "sha1-n26xgvbJqozXQwBKfU+WsZaw+ww=", + "engines": { + "node": ">=4" + } + }, + "node_modules/map-cache": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/map-cache/-/map-cache-0.2.2.tgz", + "integrity": "sha1-wyq9C9ZSXZsFFkW7TyasXcmKDb8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/map-obj": { + "version": "4.3.0", + "resolved": "https://registry.npmmirror.com/map-obj/-/map-obj-4.3.0.tgz", + "integrity": "sha512-hdN1wVrZbb29eBGiGjJbeP8JbKjq1urkHJ/LIP/NY48MZ1QVXUsQBV1G1zvYFHn1XE06cwjBsOI2K3Ulnj1YXQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/markdown-table": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/markdown-table/-/markdown-table-2.0.0.tgz", + "integrity": "sha512-Ezda85ToJUBhM6WGaG6veasyym+Tbs3cMAw/ZhOPqXiYsr0jgocBV3j3nx+4lk47plLlIqjwuTm/ywVI+zjJ/A==", + "dependencies": { + "repeat-string": "^1.0.0" + } + }, + "node_modules/md5-file": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/md5-file/-/md5-file-5.0.0.tgz", + "integrity": "sha512-xbEFXCYVWrSx/gEKS1VPlg84h/4L20znVIulKw6kMfmBUAZNAnF00eczz9ICMl+/hjQGo5KSXRxbL/47X3rmMw==", + "bin": { + "md5-file": "cli.js" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/mdast-util-definitions": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/mdast-util-definitions/-/mdast-util-definitions-4.0.0.tgz", + "integrity": "sha512-k8AJ6aNnUkB7IE+5azR9h81O5EQ/cTDXtWdMq9Kk5KcEW/8ritU5CeLg/9HhOC++nALHBlaogJ5jz0Ybk3kPMQ==", + "dependencies": { + "unist-util-visit": "^2.0.0" + } + }, + "node_modules/mdast-util-find-and-replace": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/mdast-util-find-and-replace/-/mdast-util-find-and-replace-1.1.1.tgz", + "integrity": "sha512-9cKl33Y21lyckGzpSmEQnIDjEfeeWelN5s1kUW1LwdB0Fkuq2u+4GdqcGEygYxJE8GVqCl0741bYXHgamfWAZA==", + "dependencies": { + "escape-string-regexp": "^4.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit-parents": "^3.0.0" + } + }, + "node_modules/mdast-util-find-and-replace/node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/mdast-util-footnote": { + "version": "0.1.7", + "resolved": "https://registry.npmmirror.com/mdast-util-footnote/-/mdast-util-footnote-0.1.7.tgz", + "integrity": "sha512-QxNdO8qSxqbO2e3m09KwDKfWiLgqyCurdWTQ198NpbZ2hxntdc+VKS4fDJCmNWbAroUdYnSthu+XbZ8ovh8C3w==", + "dependencies": { + "mdast-util-to-markdown": "^0.6.0", + "micromark": "~2.11.0" + } + }, + "node_modules/mdast-util-from-markdown": { + "version": "0.8.5", + "resolved": "https://registry.npmmirror.com/mdast-util-from-markdown/-/mdast-util-from-markdown-0.8.5.tgz", + "integrity": "sha512-2hkTXtYYnr+NubD/g6KGBS/0mFmBcifAsI0yIWRiRo0PjVs6SSOSOdtzbp6kSGnShDN6G5aWZpKQ2lWRy27mWQ==", + "dependencies": { + "@types/mdast": "^3.0.0", + "mdast-util-to-string": "^2.0.0", + "micromark": "~2.11.0", + "parse-entities": "^2.0.0", + "unist-util-stringify-position": "^2.0.0" + } + }, + "node_modules/mdast-util-gfm": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm/-/mdast-util-gfm-0.1.2.tgz", + "integrity": "sha512-NNkhDx/qYcuOWB7xHUGWZYVXvjPFFd6afg6/e2g+SV4r9q5XUcCbV4Wfa3DLYIiD+xAEZc6K4MGaE/m0KDcPwQ==", + "dependencies": { + "mdast-util-gfm-autolink-literal": "^0.1.0", + "mdast-util-gfm-strikethrough": "^0.2.0", + "mdast-util-gfm-table": "^0.1.0", + "mdast-util-gfm-task-list-item": "^0.1.0", + "mdast-util-to-markdown": "^0.6.1" + } + }, + "node_modules/mdast-util-gfm-autolink-literal": { + "version": "0.1.3", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-autolink-literal/-/mdast-util-gfm-autolink-literal-0.1.3.tgz", + "integrity": "sha512-GjmLjWrXg1wqMIO9+ZsRik/s7PLwTaeCHVB7vRxUwLntZc8mzmTsLVr6HW1yLokcnhfURsn5zmSVdi3/xWWu1A==", + "dependencies": { + "ccount": "^1.0.0", + "mdast-util-find-and-replace": "^1.1.0", + "micromark": "^2.11.3" + } + }, + "node_modules/mdast-util-gfm-strikethrough": { + "version": "0.2.3", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-strikethrough/-/mdast-util-gfm-strikethrough-0.2.3.tgz", + "integrity": "sha512-5OQLXpt6qdbttcDG/UxYY7Yjj3e8P7X16LzvpX8pIQPYJ/C2Z1qFGMmcw+1PZMUM3Z8wt8NRfYTvCni93mgsgA==", + "dependencies": { + "mdast-util-to-markdown": "^0.6.0" + } + }, + "node_modules/mdast-util-gfm-table": { + "version": "0.1.6", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-table/-/mdast-util-gfm-table-0.1.6.tgz", + "integrity": "sha512-j4yDxQ66AJSBwGkbpFEp9uG/LS1tZV3P33fN1gkyRB2LoRL+RR3f76m0HPHaby6F4Z5xr9Fv1URmATlRRUIpRQ==", + "dependencies": { + "markdown-table": "^2.0.0", + "mdast-util-to-markdown": "~0.6.0" + } + }, + "node_modules/mdast-util-gfm-task-list-item": { + "version": "0.1.6", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-task-list-item/-/mdast-util-gfm-task-list-item-0.1.6.tgz", + "integrity": "sha512-/d51FFIfPsSmCIRNp7E6pozM9z1GYPIkSy1urQ8s/o4TC22BZ7DqfHFWiqBD23bc7J3vV1Fc9O4QIHBlfuit8A==", + "dependencies": { + "mdast-util-to-markdown": "~0.6.0" + } + }, + "node_modules/mdast-util-to-hast": { + "version": "10.2.0", + "resolved": "https://registry.npmmirror.com/mdast-util-to-hast/-/mdast-util-to-hast-10.2.0.tgz", + "integrity": "sha512-JoPBfJ3gBnHZ18icCwHR50orC9kNH81tiR1gs01D8Q5YpV6adHNO9nKNuFBCJQ941/32PT1a63UF/DitmS3amQ==", + "dependencies": { + "@types/mdast": "^3.0.0", + "@types/unist": "^2.0.0", + "mdast-util-definitions": "^4.0.0", + "mdurl": "^1.0.0", + "unist-builder": "^2.0.0", + "unist-util-generated": "^1.0.0", + "unist-util-position": "^3.0.0", + "unist-util-visit": "^2.0.0" + } + }, + "node_modules/mdast-util-to-markdown": { + "version": "0.6.5", + "resolved": "https://registry.npmmirror.com/mdast-util-to-markdown/-/mdast-util-to-markdown-0.6.5.tgz", + "integrity": "sha512-XeV9sDE7ZlOQvs45C9UKMtfTcctcaj/pGwH8YLbMHoMOXNNCn2LsqVQOqrF1+/NU8lKDAqozme9SCXWyo9oAcQ==", + "dependencies": { + "@types/unist": "^2.0.0", + "longest-streak": "^2.0.0", + "mdast-util-to-string": "^2.0.0", + "parse-entities": "^2.0.0", + "repeat-string": "^1.0.0", + "zwitch": "^1.0.0" + } + }, + "node_modules/mdast-util-to-nlcst": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/mdast-util-to-nlcst/-/mdast-util-to-nlcst-4.0.1.tgz", + "integrity": "sha512-Y4ffygj85MTt70STKnEquw6k73jYWJBaYcb4ITAKgSNokZF7fH8rEHZ1GsRY/JaxqUevMaEnsDmkVv5Z9uVRdg==", + "dependencies": { + "nlcst-to-string": "^2.0.0", + "repeat-string": "^1.0.0", + "unist-util-position": "^3.0.0", + "vfile-location": "^3.1.0" + } + }, + "node_modules/mdast-util-to-string": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/mdast-util-to-string/-/mdast-util-to-string-2.0.0.tgz", + "integrity": "sha512-AW4DRS3QbBayY/jJmD8437V1Gombjf8RSOUCMFBuo5iHi58AGEgVCKQ+ezHkZZDpAQS75hcBMpLqjpJTjtUL7w==" + }, + "node_modules/mdast-util-toc": { + "version": "5.1.0", + "resolved": "https://registry.npmmirror.com/mdast-util-toc/-/mdast-util-toc-5.1.0.tgz", + "integrity": "sha512-csimbRIVkiqc+PpFeKDGQ/Ck2N4f9FYH3zzBMMJzcxoKL8m+cM0n94xXm0I9eaxHnKdY9n145SGTdyJC7i273g==", + "dependencies": { + "@types/mdast": "^3.0.3", + "@types/unist": "^2.0.3", + "extend": "^3.0.2", + "github-slugger": "^1.2.1", + "mdast-util-to-string": "^2.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit": "^2.0.0" + } + }, + "node_modules/mdn-data": { + "version": "2.0.14", + "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-2.0.14.tgz", + "integrity": "sha512-dn6wd0uw5GsdswPFfsgMp5NSB0/aDe6fK94YJV/AJDYXL6HVLWBsxeq7js7Ad+mU2K9LAlwpk6kN2D5mwCPVow==" + }, + "node_modules/mdurl": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/mdurl/-/mdurl-1.0.1.tgz", + "integrity": "sha512-/sKlQJCBYVY9Ers9hqzKou4H6V5UWc/M59TH2dvkt+84itfnq7uFOMLpOiOS4ujvHP4etln18fmIxA5R5fll0g==" + }, + "node_modules/meant": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/meant/-/meant-1.0.3.tgz", + "integrity": "sha512-88ZRGcNxAq4EH38cQ4D85PM57pikCwS8Z99EWHODxN7KBY+UuPiqzRTtZzS8KTXO/ywSWbdjjJST2Hly/EQxLw==" + }, + "node_modules/media-typer": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/media-typer/-/media-typer-0.3.0.tgz", + "integrity": "sha1-hxDXrwqmJvj/+hzgAWhUUmMlV0g=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/mem": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/mem/-/mem-8.1.1.tgz", + "integrity": "sha512-qFCFUDs7U3b8mBDPyz5EToEKoAkgCzqquIgi9nkkR9bixxOVOre+09lbuH7+9Kn2NFpm56M3GUWVbU2hQgdACA==", + "dependencies": { + "map-age-cleaner": "^0.1.3", + "mimic-fn": "^3.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/mem/node_modules/mimic-fn": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-3.1.0.tgz", + "integrity": "sha512-Ysbi9uYW9hFyfrThdDEQuykN4Ey6BuwPD2kpI5ES/nFTDn/98yxYNLZJcgUAKPT/mcrLLKaGzJR9YVxJrIdASQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/memfs": { + "version": "3.4.1", + "resolved": "https://registry.npmjs.org/memfs/-/memfs-3.4.1.tgz", + "integrity": "sha512-1c9VPVvW5P7I85c35zAdEr1TD5+F11IToIHIlrVIcflfnzPkJa0ZoYEoEdYDP8KgPFoSZ/opDrUsAoZWym3mtw==", + "dependencies": { + "fs-monkey": "1.0.3" + }, + "engines": { + "node": ">= 4.0.0" + } + }, + "node_modules/memoizee": { + "version": "0.4.15", + "resolved": "https://registry.npmjs.org/memoizee/-/memoizee-0.4.15.tgz", + "integrity": "sha512-UBWmJpLZd5STPm7PMUlOw/TSy972M+z8gcyQ5veOnSDRREz/0bmpyTfKt3/51DhEBqCZQn1udM/5flcSPYhkdQ==", + "dependencies": { + "d": "^1.0.1", + "es5-ext": "^0.10.53", + "es6-weak-map": "^2.0.3", + "event-emitter": "^0.3.5", + "is-promise": "^2.2.2", + "lru-queue": "^0.1.0", + "next-tick": "^1.1.0", + "timers-ext": "^0.1.7" + } + }, + "node_modules/memoizee/node_modules/is-promise": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/is-promise/-/is-promise-2.2.2.tgz", + "integrity": "sha512-+lP4/6lKUBfQjZ2pdxThZvLUAafmZb8OAxFb8XXtiQmS35INgr85hdOGoEs124ez1FCnZJt6jau/T+alh58QFQ==" + }, + "node_modules/meow": { + "version": "9.0.0", + "resolved": "https://registry.npmmirror.com/meow/-/meow-9.0.0.tgz", + "integrity": "sha512-+obSblOQmRhcyBt62furQqRAQpNyWXo8BuQ5bN7dG8wmwQ+vwHKp/rCFD4CrTP8CsDQD1sjoZ94K417XEUk8IQ==", + "dependencies": { + "@types/minimist": "^1.2.0", + "camelcase-keys": "^6.2.2", + "decamelize": "^1.2.0", + "decamelize-keys": "^1.1.0", + "hard-rejection": "^2.1.0", + "minimist-options": "4.1.0", + "normalize-package-data": "^3.0.0", + "read-pkg-up": "^7.0.1", + "redent": "^3.0.0", + "trim-newlines": "^3.0.0", + "type-fest": "^0.18.0", + "yargs-parser": "^20.2.3" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/meow/node_modules/type-fest": { + "version": "0.18.1", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.18.1.tgz", + "integrity": "sha512-OIAYXk8+ISY+qTOwkHtKqzAuxchoMiD9Udx+FSGQDuiRR+PJKJHc2NJAXlbhkGwTt/4/nKZxELY1w3ReWOL8mw==", + "engines": { + "node": ">=10" + } + }, + "node_modules/meow/node_modules/yargs-parser": { + "version": "20.2.9", + "resolved": "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-20.2.9.tgz", + "integrity": "sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w==", + "engines": { + "node": ">=10" + } + }, + "node_modules/merge-descriptors": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/merge-descriptors/-/merge-descriptors-1.0.1.tgz", + "integrity": "sha1-sAqqVW3YtEVoFQ7J0blT8/kMu2E=" + }, + "node_modules/merge-stream": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/merge-stream/-/merge-stream-2.0.0.tgz", + "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==" + }, + "node_modules/merge2": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/merge2/-/merge2-1.4.1.tgz", + "integrity": "sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg==", + "engines": { + "node": ">= 8" + } + }, + "node_modules/meros": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/meros/-/meros-1.1.4.tgz", + "integrity": "sha512-E9ZXfK9iQfG9s73ars9qvvvbSIkJZF5yOo9j4tcwM5tN8mUKfj/EKN5PzOr3ZH0y5wL7dLAHw3RVEfpQV9Q7VQ==", + "engines": { + "node": ">=12" + }, + "peerDependencies": { + "@types/node": ">=12" + }, + "peerDependenciesMeta": { + "@types/node": { + "optional": true + } + } + }, + "node_modules/methods": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/methods/-/methods-1.1.2.tgz", + "integrity": "sha1-VSmk1nZUE07cxSZmVoNbD4Ua/O4=", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/micromark": { + "version": "2.11.4", + "resolved": "https://registry.npmmirror.com/micromark/-/micromark-2.11.4.tgz", + "integrity": "sha512-+WoovN/ppKolQOFIAajxi7Lu9kInbPxFuTBVEavFcL8eAfVstoc5MocPmqBeAdBOJV00uaVjegzH4+MA0DN/uA==", + "dependencies": { + "debug": "^4.0.0", + "parse-entities": "^2.0.0" + } + }, + "node_modules/micromark-extension-footnote": { + "version": "0.3.2", + "resolved": "https://registry.npmmirror.com/micromark-extension-footnote/-/micromark-extension-footnote-0.3.2.tgz", + "integrity": "sha512-gr/BeIxbIWQoUm02cIfK7mdMZ/fbroRpLsck4kvFtjbzP4yi+OPVbnukTc/zy0i7spC2xYE/dbX1Sur8BEDJsQ==", + "dependencies": { + "micromark": "~2.11.0" + } + }, + "node_modules/micromark-extension-gfm": { + "version": "0.3.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm/-/micromark-extension-gfm-0.3.3.tgz", + "integrity": "sha512-oVN4zv5/tAIA+l3GbMi7lWeYpJ14oQyJ3uEim20ktYFAcfX1x3LNlFGGlmrZHt7u9YlKExmyJdDGaTt6cMSR/A==", + "dependencies": { + "micromark": "~2.11.0", + "micromark-extension-gfm-autolink-literal": "~0.5.0", + "micromark-extension-gfm-strikethrough": "~0.6.5", + "micromark-extension-gfm-table": "~0.4.0", + "micromark-extension-gfm-tagfilter": "~0.3.0", + "micromark-extension-gfm-task-list-item": "~0.3.0" + } + }, + "node_modules/micromark-extension-gfm-autolink-literal": { + "version": "0.5.7", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-autolink-literal/-/micromark-extension-gfm-autolink-literal-0.5.7.tgz", + "integrity": "sha512-ePiDGH0/lhcngCe8FtH4ARFoxKTUelMp4L7Gg2pujYD5CSMb9PbblnyL+AAMud/SNMyusbS2XDSiPIRcQoNFAw==", + "dependencies": { + "micromark": "~2.11.3" + } + }, + "node_modules/micromark-extension-gfm-strikethrough": { + "version": "0.6.5", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-strikethrough/-/micromark-extension-gfm-strikethrough-0.6.5.tgz", + "integrity": "sha512-PpOKlgokpQRwUesRwWEp+fHjGGkZEejj83k9gU5iXCbDG+XBA92BqnRKYJdfqfkrRcZRgGuPuXb7DaK/DmxOhw==", + "dependencies": { + "micromark": "~2.11.0" + } + }, + "node_modules/micromark-extension-gfm-table": { + "version": "0.4.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-table/-/micromark-extension-gfm-table-0.4.3.tgz", + "integrity": "sha512-hVGvESPq0fk6ALWtomcwmgLvH8ZSVpcPjzi0AjPclB9FsVRgMtGZkUcpE0zgjOCFAznKepF4z3hX8z6e3HODdA==", + "dependencies": { + "micromark": "~2.11.0" + } + }, + "node_modules/micromark-extension-gfm-tagfilter": { + "version": "0.3.0", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-tagfilter/-/micromark-extension-gfm-tagfilter-0.3.0.tgz", + "integrity": "sha512-9GU0xBatryXifL//FJH+tAZ6i240xQuFrSL7mYi8f4oZSbc+NvXjkrHemeYP0+L4ZUT+Ptz3b95zhUZnMtoi/Q==" + }, + "node_modules/micromark-extension-gfm-task-list-item": { + "version": "0.3.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-task-list-item/-/micromark-extension-gfm-task-list-item-0.3.3.tgz", + "integrity": "sha512-0zvM5iSLKrc/NQl84pZSjGo66aTGd57C1idmlWmE87lkMcXrTxg1uXa/nXomxJytoje9trP0NDLvw4bZ/Z/XCQ==", + "dependencies": { + "micromark": "~2.11.0" + } + }, + "node_modules/micromark/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmmirror.com/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/micromatch": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.5.tgz", + "integrity": "sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA==", + "dependencies": { + "braces": "^3.0.2", + "picomatch": "^2.3.1" + }, + "engines": { + "node": ">=8.6" + } + }, + "node_modules/mime": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-2.6.0.tgz", + "integrity": "sha512-USPkMeET31rOMiarsBNIHZKLGgvKc/LrjofAnBlOttf5ajRvqiRA8QsenbcooctK6d6Ts6aqZXBA+XbkKthiQg==", + "bin": { + "mime": "cli.js" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/mime-db": { + "version": "1.52.0", + "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz", + "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/mime-types": { + "version": "2.1.35", + "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz", + "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==", + "dependencies": { + "mime-db": "1.52.0" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/mimic-fn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-2.1.0.tgz", + "integrity": "sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==", + "engines": { + "node": ">=6" + } + }, + "node_modules/mimic-response": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/mimic-response/-/mimic-response-1.0.1.tgz", + "integrity": "sha512-j5EctnkH7amfV/q5Hgmoal1g2QHFJRraOtmx0JpIqkxhBhI/lJSl1nMpQ45hVarwNETOoWEimndZ4QK0RHxuxQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/min-document": { + "version": "2.19.0", + "resolved": "https://registry.npmjs.org/min-document/-/min-document-2.19.0.tgz", + "integrity": "sha1-e9KC4/WELtKVu3SM3Z8f+iyCRoU=", + "dependencies": { + "dom-walk": "^0.1.0" + } + }, + "node_modules/min-indent": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/min-indent/-/min-indent-1.0.1.tgz", + "integrity": "sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg==", + "engines": { + "node": ">=4" + } + }, + "node_modules/mini-css-extract-plugin": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/mini-css-extract-plugin/-/mini-css-extract-plugin-1.6.2.tgz", + "integrity": "sha512-WhDvO3SjGm40oV5y26GjMJYjd2UMqrLAGKy5YS2/3QKJy2F7jgynuHTir/tgUUOiNQu5saXHdc8reo7YuhhT4Q==", + "dependencies": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0", + "webpack-sources": "^1.1.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.4.0 || ^5.0.0" + } + }, + "node_modules/mini-css-extract-plugin/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/mini-svg-data-uri": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/mini-svg-data-uri/-/mini-svg-data-uri-1.4.4.tgz", + "integrity": "sha512-r9deDe9p5FJUPZAk3A59wGH7Ii9YrjjWw0jmw/liSbHl2CHiyXj6FcDXDu2K3TjVAXqiJdaw3xxwlZZr9E6nHg==", + "bin": { + "mini-svg-data-uri": "cli.js" + } + }, + "node_modules/minimatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", + "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, + "node_modules/minimist": { + "version": "1.2.6", + "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.6.tgz", + "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==" + }, + "node_modules/minimist-options": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/minimist-options/-/minimist-options-4.1.0.tgz", + "integrity": "sha512-Q4r8ghd80yhO/0j1O3B2BjweX3fiHg9cdOwjJd2J76Q135c+NDxGCqdYKQ1SKBuFfgWbAUzBfvYjPUEeNgqN1A==", + "dependencies": { + "arrify": "^1.0.1", + "is-plain-obj": "^1.1.0", + "kind-of": "^6.0.3" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/minimist-options/node_modules/arrify": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/arrify/-/arrify-1.0.1.tgz", + "integrity": "sha512-3CYzex9M9FGQjCGMGyi6/31c8GJbgb0qGyrx5HWxPd0aCwh4cB2YjMb2Xf9UuoogrMrlO9cTqnB5rI5GHZTcUA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/minipass": { + "version": "3.1.6", + "resolved": "https://registry.npmmirror.com/minipass/-/minipass-3.1.6.tgz", + "integrity": "sha512-rty5kpw9/z8SX9dmxblFA6edItUmwJgMeYDZRrwlIVN27i8gysGbznJwUggw2V/FVqFSDdWy040ZPS811DYAqQ==", + "dependencies": { + "yallist": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/minizlib": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/minizlib/-/minizlib-2.1.2.tgz", + "integrity": "sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg==", + "dependencies": { + "minipass": "^3.0.0", + "yallist": "^4.0.0" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/mitt": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/mitt/-/mitt-1.2.0.tgz", + "integrity": "sha512-r6lj77KlwqLhIUku9UWYes7KJtsczvolZkzp8hbaDPPaE24OmWl5s539Mytlj22siEQKosZ26qCBgda2PKwoJw==" + }, + "node_modules/mkdirp": { + "version": "0.5.6", + "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-0.5.6.tgz", + "integrity": "sha512-FP+p8RB8OWpF3YZBCrP5gtADmtXApB5AMLn+vdyA+PyxCjrCs00mjyUozssO33cwDeT3wNGdLxJ5M//YqtHAJw==", + "dependencies": { + "minimist": "^1.2.6" + }, + "bin": { + "mkdirp": "bin/cmd.js" + } + }, + "node_modules/mkdirp-classic": { + "version": "0.5.3", + "resolved": "https://registry.npmjs.org/mkdirp-classic/-/mkdirp-classic-0.5.3.tgz", + "integrity": "sha512-gKLcREMhtuZRwRAfqP3RFW+TK4JqApVBtOIftVgjuABpAtpxhPGaDcfvbhNvD0B8iD1oUr/txX35NjcaY6Ns/A==" + }, + "node_modules/moment": { + "version": "2.29.3", + "resolved": "https://registry.npmjs.org/moment/-/moment-2.29.3.tgz", + "integrity": "sha512-c6YRvhEo//6T2Jz/vVtYzqBzwvPT95JBQ+smCytzf7c50oMZRsR/a4w88aD34I+/QVSfnoAnSBFPJHItlOMJVw==", + "engines": { + "node": "*" + } + }, + "node_modules/ms": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", + "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" + }, + "node_modules/msgpackr": { + "version": "1.5.7", + "resolved": "https://registry.npmjs.org/msgpackr/-/msgpackr-1.5.7.tgz", + "integrity": "sha512-Hsa80i8W4BiObSMHslfnwC+CC1CYHZzoXJZn0+3EvoCEOgt3c5QlXhdcjgFk2aZxMgpV8aUFZqJyQUCIp4UrzA==", + "optionalDependencies": { + "msgpackr-extract": "^1.1.4" + } + }, + "node_modules/msgpackr-extract": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/msgpackr-extract/-/msgpackr-extract-1.1.4.tgz", + "integrity": "sha512-WQbHvsThprXh+EqZYy+SQFEs7z6bNM7a0vgirwUfwUcphWGT2mdPcpyLCNiRsN6w5q5VKJUMblHY+tNEyceb9Q==", + "hasInstallScript": true, + "optional": true, + "dependencies": { + "node-gyp-build-optional-packages": "^4.3.2" + }, + "optionalDependencies": { + "msgpackr-extract-darwin-arm64": "1.1.0", + "msgpackr-extract-darwin-x64": "1.1.0", + "msgpackr-extract-linux-arm": "1.1.0", + "msgpackr-extract-linux-arm64": "1.1.0", + "msgpackr-extract-linux-x64": "1.1.0", + "msgpackr-extract-win32-x64": "1.1.0" + } + }, + "node_modules/msgpackr-extract-darwin-arm64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-darwin-arm64/-/msgpackr-extract-darwin-arm64-1.1.0.tgz", + "integrity": "sha512-s1kHoT12tS2cCQOv+Wl3I+/cYNJXBPtwQqGA+dPYoXmchhXiE0Nso+BIfvQ5PxbmAyjj54Q5o7PnLTqVquNfZA==", + "cpu": [ + "arm64" + ], + "optional": true, + "os": [ + "darwin" + ] + }, + "node_modules/msgpackr-extract-darwin-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-darwin-x64/-/msgpackr-extract-darwin-x64-1.1.0.tgz", + "integrity": "sha512-yx/H/i12IKg4eWGu/eKdKzJD4jaYvvujQSaVmeOMCesbSQnWo5X6YR9TFjoiNoU9Aexk1KufzL9gW+1DozG1yw==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "darwin" + ] + }, + "node_modules/msgpackr-extract-linux-arm": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-arm/-/msgpackr-extract-linux-arm-1.1.0.tgz", + "integrity": "sha512-0VvSCqi12xpavxl14gMrauwIzHqHbmSChUijy/uo3mpjB1Pk4vlisKpZsaOZvNJyNKj0ACi5jYtbWnnOd7hYGw==", + "cpu": [ + "arm" + ], + "optional": true, + "os": [ + "linux" + ] + }, + "node_modules/msgpackr-extract-linux-arm64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-arm64/-/msgpackr-extract-linux-arm64-1.1.0.tgz", + "integrity": "sha512-AxFle3fHNwz2V4CYDIGFxI6o/ZuI0lBKg0uHI8EcCMUmDE5mVAUWYge5WXmORVvb8sVWyVgFlmi3MTu4Ve6tNQ==", + "cpu": [ + "arm64" + ], + "optional": true, + "os": [ + "linux" + ] + }, + "node_modules/msgpackr-extract-linux-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-x64/-/msgpackr-extract-linux-x64-1.1.0.tgz", + "integrity": "sha512-O+XoyNFWpdB8oQL6O/YyzffPpmG5rTNrr1nKLW70HD2ENJUhcITzbV7eZimHPzkn8LAGls1tBaMTHQezTBpFOw==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "linux" + ] + }, + "node_modules/msgpackr-extract-win32-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-win32-x64/-/msgpackr-extract-win32-x64-1.1.0.tgz", + "integrity": "sha512-6AJdM5rNsL4yrskRfhujVSPEd6IBpgvsnIT/TPowKNLQ62iIdryizPY2PJNFiW3AJcY249AHEiDBXS1cTDPxzA==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "win32" + ] + }, + "node_modules/msgpackr-extract/node_modules/node-gyp-build-optional-packages": { + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/node-gyp-build-optional-packages/-/node-gyp-build-optional-packages-4.3.2.tgz", + "integrity": "sha512-P5Ep3ISdmwcCkZIaBaQamQtWAG0facC89phWZgi5Z3hBU//J6S48OIvyZWSPPf6yQMklLZiqoosWAZUj7N+esA==", + "optional": true, + "bin": { + "node-gyp-build-optional": "optional.js", + "node-gyp-build-optional-packages": "bin.js", + "node-gyp-build-test": "build-test.js" + } + }, + "node_modules/multer": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/multer/-/multer-1.4.4.tgz", + "integrity": "sha512-2wY2+xD4udX612aMqMcB8Ws2Voq6NIUPEtD1be6m411T4uDH/VtL9i//xvcyFlTVfRdaBsk7hV5tgrGQqhuBiw==", + "dependencies": { + "append-field": "^1.0.0", + "busboy": "^0.2.11", + "concat-stream": "^1.5.2", + "mkdirp": "^0.5.4", + "object-assign": "^4.1.1", + "on-finished": "^2.3.0", + "type-is": "^1.6.4", + "xtend": "^4.0.0" + }, + "engines": { + "node": ">= 0.10.0" + } + }, + "node_modules/multipipe": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/multipipe/-/multipipe-1.0.2.tgz", + "integrity": "sha512-6uiC9OvY71vzSGX8lZvSqscE7ft9nPupJ8fMjrCNRAUy2LREUW42UL+V/NTrogr6rFgRydUrCX4ZitfpSNkSCQ==", + "dependencies": { + "duplexer2": "^0.1.2", + "object-assign": "^4.1.0" + } + }, + "node_modules/mute-stream": { + "version": "0.0.8", + "resolved": "https://registry.npmjs.org/mute-stream/-/mute-stream-0.0.8.tgz", + "integrity": "sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA==" + }, + "node_modules/nan": { + "version": "2.15.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.15.0.tgz", + "integrity": "sha512-8ZtvEnA2c5aYCZYd1cvgdnU6cqwixRoYg70xPLWUws5ORTa/lnw+u4amixRS/Ac5U5mQVgp9pnlSUnbNWFaWZQ==" + }, + "node_modules/nanoid": { + "version": "3.3.4", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.4.tgz", + "integrity": "sha512-MqBkQh/OHTS2egovRtLk45wEyNXwF+cokD+1YPf9u5VfJiRdAiRwB2froX5Co9Rh20xs4siNPm8naNotSD6RBw==", + "bin": { + "nanoid": "bin/nanoid.cjs" + }, + "engines": { + "node": "^10 || ^12 || ^13.7 || ^14 || >=15.0.1" + } + }, + "node_modules/napi-build-utils": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/napi-build-utils/-/napi-build-utils-1.0.2.tgz", + "integrity": "sha512-ONmRUqK7zj7DWX0D9ADe03wbwOBZxNAfF20PlGfCWQcD3+/MakShIHrMqx9YwPTfxDdF1zLeL+RGZiR9kGMLdg==" + }, + "node_modules/native-url": { + "version": "0.2.6", + "resolved": "https://registry.npmjs.org/native-url/-/native-url-0.2.6.tgz", + "integrity": "sha512-k4bDC87WtgrdD362gZz6zoiXQrl40kYlBmpfmSjwRO1VU0V5ccwJTlxuE72F6m3V0vc1xOf6n3UCP9QyerRqmA==", + "dependencies": { + "querystring": "^0.2.0" + } + }, + "node_modules/natural-compare": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", + "integrity": "sha1-Sr6/7tdUHywnrPspvbvRXI1bpPc=" + }, + "node_modules/needle": { + "version": "2.9.1", + "resolved": "https://registry.npmjs.org/needle/-/needle-2.9.1.tgz", + "integrity": "sha512-6R9fqJ5Zcmf+uYaFgdIHmLwNldn5HbK8L5ybn7Uz+ylX/rnOsSp1AHcvQSrCaFN+qNM1wpymHqD7mVasEOlHGQ==", + "dependencies": { + "debug": "^3.2.6", + "iconv-lite": "^0.4.4", + "sax": "^1.2.4" + }, + "bin": { + "needle": "bin/needle" + }, + "engines": { + "node": ">= 4.4.x" + } + }, + "node_modules/negotiator": { + "version": "0.6.3", + "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.6.3.tgz", + "integrity": "sha512-+EUsqGPLsM+j/zdChZjsnX51g4XrHFOIXwfnCVPGlQk/k5giakcKsuxCObBRu6DSm9opw/O6slWbJdghQM4bBg==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/neo-async": { + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/neo-async/-/neo-async-2.6.2.tgz", + "integrity": "sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw==" + }, + "node_modules/next-tick": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/next-tick/-/next-tick-1.1.0.tgz", + "integrity": "sha512-CXdUiJembsNjuToQvxayPZF9Vqht7hewsvy2sOWafLvi2awflj9mOC6bHIg50orX8IJvWKY9wYQ/zB2kogPslQ==" + }, + "node_modules/nice-try": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/nice-try/-/nice-try-1.0.5.tgz", + "integrity": "sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ==" + }, + "node_modules/nlcst-to-string": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/nlcst-to-string/-/nlcst-to-string-2.0.4.tgz", + "integrity": "sha512-3x3jwTd6UPG7vi5k4GEzvxJ5rDA7hVUIRNHPblKuMVP9Z3xmlsd9cgLcpAMkc5uPOBna82EeshROFhsPkbnTZg==" + }, + "node_modules/no-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/no-case/-/no-case-3.0.4.tgz", + "integrity": "sha512-fgAN3jGAh+RoxUGZHTSOLJIqUc2wmoBwGR4tbpNAKmmovFoWq0OdRkb0VkldReO2a2iBT/OEulG9XSUc10r3zg==", + "dependencies": { + "lower-case": "^2.0.2", + "tslib": "^2.0.3" + } + }, + "node_modules/node-abi": { + "version": "3.15.0", + "resolved": "https://registry.npmjs.org/node-abi/-/node-abi-3.15.0.tgz", + "integrity": "sha512-Ic6z/j6I9RLm4ov7npo1I48UQr2BEyFCqh6p7S1dhEx9jPO0GPGq/e2Rb7x7DroQrmiVMz/Bw1vJm9sPAl2nxA==", + "dependencies": { + "semver": "^7.3.5" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/node-addon-api": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/node-addon-api/-/node-addon-api-3.2.1.tgz", + "integrity": "sha512-mmcei9JghVNDYydghQmeDX8KoAm0FAiYyIcUt/N4nhyAipB17pllZQDOJD2fotxABnt4Mdz+dKTO7eftLg4d0A==" + }, + "node_modules/node-fetch": { + "version": "2.6.7", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", + "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", + "dependencies": { + "whatwg-url": "^5.0.0" + }, + "engines": { + "node": "4.x || >=6.0.0" + }, + "peerDependencies": { + "encoding": "^0.1.0" + }, + "peerDependenciesMeta": { + "encoding": { + "optional": true + } + } + }, + "node_modules/node-gyp": { + "version": "7.1.2", + "resolved": "https://registry.npmmirror.com/node-gyp/-/node-gyp-7.1.2.tgz", + "integrity": "sha512-CbpcIo7C3eMu3dL1c3d0xw449fHIGALIJsRP4DDPHpyiW8vcriNY7ubh9TE4zEKfSxscY7PjeFnshE7h75ynjQ==", + "dependencies": { + "env-paths": "^2.2.0", + "glob": "^7.1.4", + "graceful-fs": "^4.2.3", + "nopt": "^5.0.0", + "npmlog": "^4.1.2", + "request": "^2.88.2", + "rimraf": "^3.0.2", + "semver": "^7.3.2", + "tar": "^6.0.2", + "which": "^2.0.2" + }, + "bin": { + "node-gyp": "bin/node-gyp.js" + }, + "engines": { + "node": ">= 10.12.0" + } + }, + "node_modules/node-gyp-build": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/node-gyp-build/-/node-gyp-build-4.4.0.tgz", + "integrity": "sha512-amJnQCcgtRVw9SvoebO3BKGESClrfXGCUTX9hSn1OuGQTQBOZmVd0Z0OlecpuRksKvbsUqALE8jls/ErClAPuQ==", + "bin": { + "node-gyp-build": "bin.js", + "node-gyp-build-optional": "optional.js", + "node-gyp-build-test": "build-test.js" + } + }, + "node_modules/node-gyp/node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmmirror.com/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/node-int64": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/node-int64/-/node-int64-0.4.0.tgz", + "integrity": "sha1-h6kGXNs1XTGC2PlM4RGIuCXGijs=" + }, + "node_modules/node-object-hash": { + "version": "2.3.10", + "resolved": "https://registry.npmjs.org/node-object-hash/-/node-object-hash-2.3.10.tgz", + "integrity": "sha512-jY5dPJzw6NHd/KPSfPKJ+IHoFS81/tJ43r34ZeNMXGzCOM8jwQDCD12HYayKIB6MuznrnqIYy2e891NA2g0ibA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/node-releases": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.4.tgz", + "integrity": "sha512-gbMzqQtTtDz/00jQzZ21PQzdI9PyLYqUSvD0p3naOhX4odFji0ZxYdnVwPTxmSwkmxhcFImpozceidSG+AgoPQ==" + }, + "node_modules/node-sass": { + "version": "6.0.1", + "resolved": "https://registry.npmmirror.com/node-sass/-/node-sass-6.0.1.tgz", + "integrity": "sha512-f+Rbqt92Ful9gX0cGtdYwjTrWAaGURgaK5rZCWOgCNyGWusFYHhbqCCBoFBeat+HKETOU02AyTxNhJV0YZf2jQ==", + "hasInstallScript": true, + "dependencies": { + "async-foreach": "^0.1.3", + "chalk": "^1.1.1", + "cross-spawn": "^7.0.3", + "gaze": "^1.0.0", + "get-stdin": "^4.0.1", + "glob": "^7.0.3", + "lodash": "^4.17.15", + "meow": "^9.0.0", + "nan": "^2.13.2", + "node-gyp": "^7.1.0", + "npmlog": "^4.0.0", + "request": "^2.88.0", + "sass-graph": "2.2.5", + "stdout-stream": "^1.4.0", + "true-case-path": "^1.0.2" + }, + "bin": { + "node-sass": "bin/node-sass" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/node-sass/node_modules/ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha512-TIGnTpdo+E3+pCyAluZvtED5p5wCqLdezCyhPZzKPcxvFplEt4i+W7OONCKgeZFT3+y5NZZfOOS/Bdcanm1MYA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/node-sass/node_modules/ansi-styles": { + "version": "2.2.1", + "resolved": "https://registry.npmmirror.com/ansi-styles/-/ansi-styles-2.2.1.tgz", + "integrity": "sha512-kmCevFghRiWM7HB5zTPULl4r9bVFSWjz62MhqizDGUrq2NWuNMQyuv4tHHoKJHs69M/MF64lEcHdYIocrdWQYA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/node-sass/node_modules/chalk": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/chalk/-/chalk-1.1.3.tgz", + "integrity": "sha512-U3lRVLMSlsCfjqYPbLyVv11M9CPW4I728d6TCKMAOJueEeB9/8o+eSsMnxPJD+Q+K909sdESg7C+tIkoH6on1A==", + "dependencies": { + "ansi-styles": "^2.2.1", + "escape-string-regexp": "^1.0.2", + "has-ansi": "^2.0.0", + "strip-ansi": "^3.0.0", + "supports-color": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/node-sass/node_modules/cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmmirror.com/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "dependencies": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/node-sass/node_modules/path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/node-sass/node_modules/shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "dependencies": { + "shebang-regex": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/node-sass/node_modules/shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", + "engines": { + "node": ">=8" + } + }, + "node_modules/node-sass/node_modules/strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha512-VhumSSbBqDTP8p2ZLKj40UjBCV4+v8bUSEpUb4KjRgWk9pbqGF4REFj6KEagidb2f/M6AzC0EmFyDNGaw9OCzg==", + "dependencies": { + "ansi-regex": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/node-sass/node_modules/supports-color": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/supports-color/-/supports-color-2.0.0.tgz", + "integrity": "sha512-KKNVtd6pCYgPIKU4cp2733HWYCpplQhddZLBUryaAHou723x+FRzQ5Df824Fj+IyyuiQTRoub4SnIFfIcrp70g==", + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/node-sass/node_modules/true-case-path": { + "version": "1.0.3", + "resolved": "https://registry.npmmirror.com/true-case-path/-/true-case-path-1.0.3.tgz", + "integrity": "sha512-m6s2OdQe5wgpFMC+pAJ+q9djG82O2jcHPOI6RNg1yy9rCYR+WD6Nbpl32fDpfC56nirdRy+opFa/Vk7HYhqaew==", + "dependencies": { + "glob": "^7.1.2" + } + }, + "node_modules/node-sass/node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmmirror.com/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/nopt": { + "version": "5.0.0", + "resolved": "https://registry.npmmirror.com/nopt/-/nopt-5.0.0.tgz", + "integrity": "sha512-Tbj67rffqceeLpcRXrT7vKAN8CwfPeIBgM7E6iBkmKLV7bEMwpGgYLGv0jACUsECaa/vuxP0IjEont6umdMgtQ==", + "dependencies": { + "abbrev": "1" + }, + "bin": { + "nopt": "bin/nopt.js" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/normalize-package-data": { + "version": "3.0.3", + "resolved": "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-3.0.3.tgz", + "integrity": "sha512-p2W1sgqij3zMMyRC067Dg16bfzVH+w7hyegmpIvZ4JNjqtGOVAIvLmjBx3yP7YTe9vKJgkoNOPjwQGogDoMXFA==", + "dependencies": { + "hosted-git-info": "^4.0.1", + "is-core-module": "^2.5.0", + "semver": "^7.3.4", + "validate-npm-package-license": "^3.0.1" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/normalize-package-data/node_modules/hosted-git-info": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-4.1.0.tgz", + "integrity": "sha512-kyCuEOWjJqZuDbRHzL8V93NzQhwIB71oFWSyzVo+KPZI+pnQPPxucdkrOZvkLRnrf5URsQM+IJ09Dw29cRALIA==", + "dependencies": { + "lru-cache": "^6.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/normalize-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", + "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/normalize-range": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/normalize-range/-/normalize-range-0.1.2.tgz", + "integrity": "sha1-LRDAa9/TEuqXd2laTShDlFa3WUI=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/normalize-url": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-6.1.0.tgz", + "integrity": "sha512-DlL+XwOy3NxAQ8xuC0okPgK46iuVNAK01YN7RueYBqqFeGsBjV9XmCAzAdgt+667bCl5kPh9EqKKDwnaPG1I7A==", + "engines": { + "node": ">=10" + } + }, + "node_modules/not": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/not/-/not-0.1.0.tgz", + "integrity": "sha512-5PDmaAsVfnWUgTUbJ3ERwn7u79Z0dYxN9ErxCpVJJqe2RK0PJ3z+iFUxuqjwtlDDegXvtWoxD/3Fzxox7tFGWA==" + }, + "node_modules/npm-run-path": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-4.0.1.tgz", + "integrity": "sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw==", + "dependencies": { + "path-key": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/npm-run-path/node_modules/path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/npmlog": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-4.1.2.tgz", + "integrity": "sha512-2uUqazuKlTaSI/dC8AzicUck7+IrEaOnN/e0jd3Xtt1KcGpwx30v50mL7oPyr/h9bL3E4aZccVwpwP+5W9Vjkg==", + "dependencies": { + "are-we-there-yet": "~1.1.2", + "console-control-strings": "~1.1.0", + "gauge": "~2.7.3", + "set-blocking": "~2.0.0" + } + }, + "node_modules/nth-check": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/nth-check/-/nth-check-2.0.1.tgz", + "integrity": "sha512-it1vE95zF6dTT9lBsYbxvqh0Soy4SPowchj0UBGj/V6cTPnXXtQOPUbhZ6CmGzAD/rW22LQK6E96pcdJXk4A4w==", + "dependencies": { + "boolbase": "^1.0.0" + } + }, + "node_modules/null-loader": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/null-loader/-/null-loader-4.0.1.tgz", + "integrity": "sha512-pxqVbi4U6N26lq+LmgIbB5XATP0VdZKOG25DhHi8btMmJJefGArFyDg1yc4U3hWCJbMqSrw0qyrz1UQX+qYXqg==", + "dependencies": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/null-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/nullthrows": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/nullthrows/-/nullthrows-1.1.1.tgz", + "integrity": "sha512-2vPPEi+Z7WqML2jZYddDIfy5Dqb0r2fze2zTxNNknZaFpVHU3mFB3R+DWeJWGVx0ecvttSGlJTI+WG+8Z4cDWw==" + }, + "node_modules/number-is-nan": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/number-is-nan/-/number-is-nan-1.0.1.tgz", + "integrity": "sha1-CXtgK1NCKlIsGvuHkDGDNpQaAR0=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/oauth-sign": { + "version": "0.9.0", + "resolved": "https://registry.npmmirror.com/oauth-sign/-/oauth-sign-0.9.0.tgz", + "integrity": "sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ==", + "engines": { + "node": "*" + } + }, + "node_modules/object-assign": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", + "integrity": "sha1-IQmtx5ZYh8/AXLvUQsrIv7s2CGM=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/object-inspect": { + "version": "1.12.0", + "resolved": "https://registry.npmjs.org/object-inspect/-/object-inspect-1.12.0.tgz", + "integrity": "sha512-Ho2z80bVIvJloH+YzRmpZVQe87+qASmBUKZDWgx9cu+KDrX2ZDH/3tMy+gXbZETVGs2M8YdxObOh7XAtim9Y0g==" + }, + "node_modules/object-keys": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", + "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==", + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/object.assign": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/object.assign/-/object.assign-4.1.2.tgz", + "integrity": "sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ==", + "dependencies": { + "call-bind": "^1.0.0", + "define-properties": "^1.1.3", + "has-symbols": "^1.0.1", + "object-keys": "^1.1.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/object.entries": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/object.entries/-/object.entries-1.1.5.tgz", + "integrity": "sha512-TyxmjUoZggd4OrrU1W66FMDG6CuqJxsFvymeyXI51+vQLN67zYfZseptRge703kKQdo4uccgAKebXFcRCzk4+g==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/object.fromentries": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/object.fromentries/-/object.fromentries-2.0.5.tgz", + "integrity": "sha512-CAyG5mWQRRiBU57Re4FKoTBjXfDoNwdFVH2Y1tS9PqCsfUTymAohOkEMSG3aRNKmv4lV3O7p1et7c187q6bynw==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/object.getownpropertydescriptors": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/object.getownpropertydescriptors/-/object.getownpropertydescriptors-2.1.3.tgz", + "integrity": "sha512-VdDoCwvJI4QdC6ndjpqFmoL3/+HxffFBbcJzKi5hwLLqqx3mdbedRpfZDdK0SrOSauj8X4GzBvnDZl4vTN7dOw==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/object.hasown": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/object.hasown/-/object.hasown-1.1.1.tgz", + "integrity": "sha512-LYLe4tivNQzq4JdaWW6WO3HMZZJWzkkH8fnI6EebWl0VZth2wL2Lovm74ep2/gZzlaTdV62JZHEqHQ2yVn8Q/A==", + "dependencies": { + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "node_modules/object.values": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/object.values/-/object.values-1.1.5.tgz", + "integrity": "sha512-QUZRW0ilQ3PnPpbNtgdNV1PDbEqLIiSFB3l+EnGtBQ/8SUTLj1PZwtQHABZtLgwpJZTSZhuGLOGk57Drx2IvYg==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/objectFitPolyfill": { + "version": "2.3.5", + "resolved": "https://registry.npmjs.org/objectFitPolyfill/-/objectFitPolyfill-2.3.5.tgz", + "integrity": "sha512-8Quz071ZmGi0QWEG4xB3Bv5Lpw6K0Uca87FLoLMKMWjB6qIq9IyBegP3b/VLNxv2WYvIMGoeUQ+c6ibUkNa8TA==" + }, + "node_modules/omggif": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/omggif/-/omggif-1.0.10.tgz", + "integrity": "sha512-LMJTtvgc/nugXj0Vcrrs68Mn2D1r0zf630VNtqtpI1FEO7e+O9FP4gqs9AcnBaSEeoHIPm28u6qgPR0oyEpGSw==" + }, + "node_modules/on-finished": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/on-finished/-/on-finished-2.4.1.tgz", + "integrity": "sha512-oVlzkg3ENAhCk2zdv7IJwd/QUD4z2RxRwpkcGY8psCVcCYZNq4wYnVWALHM+brtuJjePWiYF/ClmuDr8Ch5+kg==", + "dependencies": { + "ee-first": "1.1.1" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/on-headers": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/on-headers/-/on-headers-1.0.2.tgz", + "integrity": "sha512-pZAE+FJLoyITytdqK0U5s+FIpjN0JP3OzFi/u8Rx+EV5/W+JTWGXG8xFzevE7AjBfDqHv/8vL8qQsIhHnqRkrA==", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/once": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", + "integrity": "sha1-WDsap3WWHUsROsF9nFC6753Xa9E=", + "dependencies": { + "wrappy": "1" + } + }, + "node_modules/onetime": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/onetime/-/onetime-5.1.2.tgz", + "integrity": "sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg==", + "dependencies": { + "mimic-fn": "^2.1.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/open": { + "version": "7.4.2", + "resolved": "https://registry.npmjs.org/open/-/open-7.4.2.tgz", + "integrity": "sha512-MVHddDVweXZF3awtlAS+6pgKLlm/JgxZ90+/NBurBoQctVOOB/zDdVjcyPzQ+0laDGbsWgrRkflI65sQeOgT9Q==", + "dependencies": { + "is-docker": "^2.0.0", + "is-wsl": "^2.1.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/opentracing": { + "version": "0.14.7", + "resolved": "https://registry.npmjs.org/opentracing/-/opentracing-0.14.7.tgz", + "integrity": "sha512-vz9iS7MJ5+Bp1URw8Khvdyw1H/hGvzHWlKQ7eRrQojSCDL1/SrWfrY9QebLw97n2deyRtzHRC3MkQfVNUCo91Q==", + "engines": { + "node": ">=0.10" + } + }, + "node_modules/optionator": { + "version": "0.9.1", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.1.tgz", + "integrity": "sha512-74RlY5FCnhq4jRxVUPKDaRwrVNXMqsGsiW6AJw4XK8hmtm10wC0ypZBLw5IIp85NZMr91+qd1RvvENwg7jjRFw==", + "dependencies": { + "deep-is": "^0.1.3", + "fast-levenshtein": "^2.0.6", + "levn": "^0.4.1", + "prelude-ls": "^1.2.1", + "type-check": "^0.4.0", + "word-wrap": "^1.2.3" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/ordered-binary": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/ordered-binary/-/ordered-binary-1.2.5.tgz", + "integrity": "sha512-djRmZoEpOGvIRW7ufsCDHtvcUa18UC9TxnPbHhSVFZHsoyg0dtut1bWtBZ/fmxdPN62oWXrV6adM7NoWU+CneA==" + }, + "node_modules/os-tmpdir": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/os-tmpdir/-/os-tmpdir-1.0.2.tgz", + "integrity": "sha1-u+Z0BseaqFxc/sdm/lc0VV36EnQ=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/p-cancelable": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-2.1.1.tgz", + "integrity": "sha512-BZOr3nRQHOntUjTrH8+Lh54smKHoHyur8We1V8DSMVrl5A2malOOwuJRnKRDjSnkoeBh4at6BwEnb5I7Jl31wg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/p-defer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/p-defer/-/p-defer-3.0.0.tgz", + "integrity": "sha512-ugZxsxmtTln604yeYd29EGrNhazN2lywetzpKhfmQjW/VJmhpDmWbiX+h0zL8V91R0UXkhb3KtPmyq9PZw3aYw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/p-finally": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-finally/-/p-finally-1.0.0.tgz", + "integrity": "sha1-P7z7FbiZpEEjs0ttzBi3JDNqLK4=", + "engines": { + "node": ">=4" + } + }, + "node_modules/p-limit": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", + "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "dependencies": { + "p-try": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/p-locate": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", + "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", + "dependencies": { + "p-limit": "^2.2.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/p-try": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-2.2.0.tgz", + "integrity": "sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/package-json": { + "version": "6.5.0", + "resolved": "https://registry.npmjs.org/package-json/-/package-json-6.5.0.tgz", + "integrity": "sha512-k3bdm2n25tkyxcjSKzB5x8kfVxlMdgsbPr0GkZcwHsLpba6cBjqCt1KlcChKEvxHIcTB1FVMuwoijZ26xex5MQ==", + "dependencies": { + "got": "^9.6.0", + "registry-auth-token": "^4.0.0", + "registry-url": "^5.0.0", + "semver": "^6.2.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/package-json/node_modules/@sindresorhus/is": { + "version": "0.14.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-0.14.0.tgz", + "integrity": "sha512-9NET910DNaIPngYnLLPeg+Ogzqsi9uM4mSboU5y6p8S5DzMTVEsJZrawi+BoDNUVBa2DhJqQYUFvMDfgU062LQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/package-json/node_modules/@szmarczak/http-timer": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-1.1.2.tgz", + "integrity": "sha512-XIB2XbzHTN6ieIjfIMV9hlVcfPU26s2vafYWQcZHWXHOxiaRZYEDKEwdl129Zyg50+foYV2jCgtrqSA6qNuNSA==", + "dependencies": { + "defer-to-connect": "^1.0.1" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/package-json/node_modules/cacheable-request": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-6.1.0.tgz", + "integrity": "sha512-Oj3cAGPCqOZX7Rz64Uny2GYAZNliQSqfbePrgAQ1wKAihYmCUnraBtJtKcGR4xz7wF+LoJC+ssFZvv5BgF9Igg==", + "dependencies": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^3.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^4.1.0", + "responselike": "^1.0.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/package-json/node_modules/cacheable-request/node_modules/get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/package-json/node_modules/cacheable-request/node_modules/lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/package-json/node_modules/decompress-response": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-3.3.0.tgz", + "integrity": "sha1-gKTdMjdIOEv6JICDYirt7Jgq3/M=", + "dependencies": { + "mimic-response": "^1.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/package-json/node_modules/defer-to-connect": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-1.1.3.tgz", + "integrity": "sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ==" + }, + "node_modules/package-json/node_modules/get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "dependencies": { + "pump": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/package-json/node_modules/got": { + "version": "9.6.0", + "resolved": "https://registry.npmjs.org/got/-/got-9.6.0.tgz", + "integrity": "sha512-R7eWptXuGYxwijs0eV+v3o6+XH1IqVK8dJOEecQfTmkncw9AV4dcw/Dhxi8MdlqPthxxpZyizMzyg8RTmEsG+Q==", + "dependencies": { + "@sindresorhus/is": "^0.14.0", + "@szmarczak/http-timer": "^1.1.2", + "cacheable-request": "^6.0.0", + "decompress-response": "^3.3.0", + "duplexer3": "^0.1.4", + "get-stream": "^4.1.0", + "lowercase-keys": "^1.0.1", + "mimic-response": "^1.0.1", + "p-cancelable": "^1.0.0", + "to-readable-stream": "^1.0.0", + "url-parse-lax": "^3.0.0" + }, + "engines": { + "node": ">=8.6" + } + }, + "node_modules/package-json/node_modules/json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha1-Wx85evx11ne96Lz8Dkfh+aPZqJg=" + }, + "node_modules/package-json/node_modules/keyv": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-3.1.0.tgz", + "integrity": "sha512-9ykJ/46SN/9KPM/sichzQ7OvXyGDYKGTaDlKMGCAlg2UK8KRy4jb0d8sFc+0Tt0YYnThq8X2RZgCg74RPxgcVA==", + "dependencies": { + "json-buffer": "3.0.0" + } + }, + "node_modules/package-json/node_modules/lowercase-keys": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-1.0.1.tgz", + "integrity": "sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/package-json/node_modules/normalize-url": { + "version": "4.5.1", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-4.5.1.tgz", + "integrity": "sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/package-json/node_modules/p-cancelable": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-1.1.0.tgz", + "integrity": "sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/package-json/node_modules/responselike": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-1.0.2.tgz", + "integrity": "sha1-kYcg7ztjHFZCvgaPFa3lpG9Loec=", + "dependencies": { + "lowercase-keys": "^1.0.0" + } + }, + "node_modules/package-json/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/pako": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/pako/-/pako-1.0.11.tgz", + "integrity": "sha512-4hLB8Py4zZce5s4yd9XzopqwVv/yGNhV1Bl8NTmCq1763HeK2+EwVTv+leGeL13Dnh2wfbqowVPXCIO0z4taYw==" + }, + "node_modules/param-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/param-case/-/param-case-3.0.4.tgz", + "integrity": "sha512-RXlj7zCYokReqWpOPH9oYivUzLYZ5vAPIfEmCTNViosC78F8F0H9y7T7gG2M39ymgutxF5gcFEsyZQSph9Bp3A==", + "dependencies": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/parent-module": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/parent-module/-/parent-module-1.0.1.tgz", + "integrity": "sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g==", + "dependencies": { + "callsites": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/parse-bmfont-ascii": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/parse-bmfont-ascii/-/parse-bmfont-ascii-1.0.6.tgz", + "integrity": "sha1-Eaw8P/WPfCAgqyJ2kHkQjU36AoU=" + }, + "node_modules/parse-bmfont-binary": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/parse-bmfont-binary/-/parse-bmfont-binary-1.0.6.tgz", + "integrity": "sha1-0Di0dtPp3Z2x4RoLDlOiJ5K2kAY=" + }, + "node_modules/parse-bmfont-xml": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/parse-bmfont-xml/-/parse-bmfont-xml-1.1.4.tgz", + "integrity": "sha512-bjnliEOmGv3y1aMEfREMBJ9tfL3WR0i0CKPj61DnSLaoxWR3nLrsQrEbCId/8rF4NyRF0cCqisSVXyQYWM+mCQ==", + "dependencies": { + "xml-parse-from-string": "^1.0.0", + "xml2js": "^0.4.5" + } + }, + "node_modules/parse-english": { + "version": "4.2.0", + "resolved": "https://registry.npmmirror.com/parse-english/-/parse-english-4.2.0.tgz", + "integrity": "sha512-jw5N6wZUZViIw3VLG/FUSeL3vDhfw5Q2g4E3nYC69Mm5ANbh9ZWd+eligQbeUoyObZM8neynTn3l14e09pjEWg==", + "dependencies": { + "nlcst-to-string": "^2.0.0", + "parse-latin": "^4.0.0", + "unist-util-modify-children": "^2.0.0", + "unist-util-visit-children": "^1.0.0" + } + }, + "node_modules/parse-entities": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/parse-entities/-/parse-entities-2.0.0.tgz", + "integrity": "sha512-kkywGpCcRYhqQIchaWqZ875wzpS/bMKhz5HnN3p7wveJTkTtyAB/AlnS0f8DFSqYW1T82t6yEAkEcB+A1I3MbQ==", + "dependencies": { + "character-entities": "^1.0.0", + "character-entities-legacy": "^1.0.0", + "character-reference-invalid": "^1.0.0", + "is-alphanumerical": "^1.0.0", + "is-decimal": "^1.0.0", + "is-hexadecimal": "^1.0.0" + } + }, + "node_modules/parse-filepath": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/parse-filepath/-/parse-filepath-1.0.2.tgz", + "integrity": "sha1-pjISf1Oq89FYdvWHLz/6x2PWyJE=", + "dependencies": { + "is-absolute": "^1.0.0", + "map-cache": "^0.2.0", + "path-root": "^0.1.1" + }, + "engines": { + "node": ">=0.8" + } + }, + "node_modules/parse-headers": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/parse-headers/-/parse-headers-2.0.5.tgz", + "integrity": "sha512-ft3iAoLOB/MlwbNXgzy43SWGP6sQki2jQvAyBg/zDFAgr9bfNWZIUj42Kw2eJIl8kEi4PbgE6U1Zau/HwI75HA==" + }, + "node_modules/parse-json": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/parse-json/-/parse-json-5.2.0.tgz", + "integrity": "sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg==", + "dependencies": { + "@babel/code-frame": "^7.0.0", + "error-ex": "^1.3.1", + "json-parse-even-better-errors": "^2.3.0", + "lines-and-columns": "^1.1.6" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/parse-latin": { + "version": "4.3.0", + "resolved": "https://registry.npmmirror.com/parse-latin/-/parse-latin-4.3.0.tgz", + "integrity": "sha512-TYKL+K98dcAWoCw/Ac1yrPviU8Trk+/gmjQVaoWEFDZmVD4KRg6c/80xKqNNFQObo2mTONgF8trzAf2UTwKafw==", + "dependencies": { + "nlcst-to-string": "^2.0.0", + "unist-util-modify-children": "^2.0.0", + "unist-util-visit-children": "^1.0.0" + } + }, + "node_modules/parse-path": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/parse-path/-/parse-path-4.0.3.tgz", + "integrity": "sha512-9Cepbp2asKnWTJ9x2kpw6Fe8y9JDbqwahGCTvklzd/cEq5C5JC59x2Xb0Kx+x0QZ8bvNquGO8/BWP0cwBHzSAA==", + "dependencies": { + "is-ssh": "^1.3.0", + "protocols": "^1.4.0", + "qs": "^6.9.4", + "query-string": "^6.13.8" + } + }, + "node_modules/parse-srcset": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/parse-srcset/-/parse-srcset-1.0.2.tgz", + "integrity": "sha512-/2qh0lav6CmI15FzA3i/2Bzk2zCgQhGMkvhOhKNcBVQ1ldgpbfiNTVslmooUmWJcADi1f1kIeynbDRVzNlfR6Q==" + }, + "node_modules/parse-url": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/parse-url/-/parse-url-6.0.0.tgz", + "integrity": "sha512-cYyojeX7yIIwuJzledIHeLUBVJ6COVLeT4eF+2P6aKVzwvgKQPndCBv3+yQ7pcWjqToYwaligxzSYNNmGoMAvw==", + "dependencies": { + "is-ssh": "^1.3.0", + "normalize-url": "^6.1.0", + "parse-path": "^4.0.0", + "protocols": "^1.4.0" + } + }, + "node_modules/parse5": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/parse5/-/parse5-6.0.1.tgz", + "integrity": "sha512-Ofn/CTFzRGTTxwpNEs9PP93gXShHcTq255nzRYSKe8AkVpZY7e1fpmTfOyoIvjP5HG7Z2ZM7VS9PPhQGW2pOpw==" + }, + "node_modules/parse5-htmlparser2-tree-adapter": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/parse5-htmlparser2-tree-adapter/-/parse5-htmlparser2-tree-adapter-6.0.1.tgz", + "integrity": "sha512-qPuWvbLgvDGilKc5BoicRovlT4MtYT6JfJyBOMDsKoiT+GiuP5qyrPCnR9HcPECIJJmZh5jRndyNThnhhb/vlA==", + "dependencies": { + "parse5": "^6.0.1" + } + }, + "node_modules/parseqs": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/parseqs/-/parseqs-0.0.6.tgz", + "integrity": "sha512-jeAGzMDbfSHHA091hr0r31eYfTig+29g3GKKE/PPbEQ65X0lmMwlEoqmhzu0iztID5uJpZsFlUPDP8ThPL7M8w==" + }, + "node_modules/parseuri": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/parseuri/-/parseuri-0.0.6.tgz", + "integrity": "sha512-AUjen8sAkGgao7UyCX6Ahv0gIK2fABKmYjvP4xmy5JaKvcbTRueIqIPHLAfq30xJddqSE033IOMUSOMCcK3Sow==" + }, + "node_modules/parseurl": { + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/parseurl/-/parseurl-1.3.3.tgz", + "integrity": "sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ==", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/pascal-case": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/pascal-case/-/pascal-case-3.1.2.tgz", + "integrity": "sha512-uWlGT3YSnK9x3BQJaOdcZwrnV6hPpd8jFH1/ucpiLRPh/2zCVJKS19E4GvYHvaCcACn3foXZ0cLB9Wrx1KGe5g==", + "dependencies": { + "no-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/password-prompt": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/password-prompt/-/password-prompt-1.1.2.tgz", + "integrity": "sha512-bpuBhROdrhuN3E7G/koAju0WjVw9/uQOG5Co5mokNj0MiOSBVZS1JTwM4zl55hu0WFmIEFvO9cU9sJQiBIYeIA==", + "dependencies": { + "ansi-escapes": "^3.1.0", + "cross-spawn": "^6.0.5" + } + }, + "node_modules/path-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/path-case/-/path-case-3.0.4.tgz", + "integrity": "sha512-qO4qCFjXqVTrcbPt/hQfhTQ+VhFsqNKOPtytgNKkKxSoEp3XPUQ8ObFuePylOIok5gjn69ry8XiULxCwot3Wfg==", + "dependencies": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/path-exists": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", + "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", + "engines": { + "node": ">=8" + } + }, + "node_modules/path-is-absolute": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", + "integrity": "sha1-F0uSaHNVNP+8es5r9TpanhtcX18=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/path-key": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-2.0.1.tgz", + "integrity": "sha1-QRyttXTFoUDTpLGRDUDYDMn0C0A=", + "engines": { + "node": ">=4" + } + }, + "node_modules/path-parse": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.7.tgz", + "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==" + }, + "node_modules/path-root": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/path-root/-/path-root-0.1.1.tgz", + "integrity": "sha1-mkpoFMrBwM1zNgqV8yCDyOpHRbc=", + "dependencies": { + "path-root-regex": "^0.1.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/path-root-regex": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/path-root-regex/-/path-root-regex-0.1.2.tgz", + "integrity": "sha1-v8zcjfWxLcUsi0PsONGNcsBLqW0=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/path-to-regexp": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-0.1.7.tgz", + "integrity": "sha1-32BBeABfUi8V60SQ5yR6G/qmf4w=" + }, + "node_modules/path-type": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", + "integrity": "sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/peek-readable": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/peek-readable/-/peek-readable-4.1.0.tgz", + "integrity": "sha512-ZI3LnwUv5nOGbQzD9c2iDG6toheuXSZP5esSHBjopsXH4dg19soufvpUGA3uohi5anFtGb2lhAVdHzH6R/Evvg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/performance-now": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/performance-now/-/performance-now-2.1.0.tgz", + "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==" + }, + "node_modules/phin": { + "version": "2.9.3", + "resolved": "https://registry.npmjs.org/phin/-/phin-2.9.3.tgz", + "integrity": "sha512-CzFr90qM24ju5f88quFC/6qohjC144rehe5n6DH900lgXmUe86+xCKc10ev56gRKC4/BkHUoG4uSiQgBiIXwDA==" + }, + "node_modules/physical-cpu-count": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/physical-cpu-count/-/physical-cpu-count-2.0.0.tgz", + "integrity": "sha1-GN4vl+S/epVRrXURlCtUlverpmA=" + }, + "node_modules/picocolors": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.0.tgz", + "integrity": "sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ==" + }, + "node_modules/picomatch": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", + "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", + "engines": { + "node": ">=8.6" + } + }, + "node_modules/pixelmatch": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/pixelmatch/-/pixelmatch-4.0.2.tgz", + "integrity": "sha1-j0fc7FARtHe2fbA8JDvB8wheiFQ=", + "dependencies": { + "pngjs": "^3.0.0" + }, + "bin": { + "pixelmatch": "bin/pixelmatch" + } + }, + "node_modules/pkg-dir": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/pkg-dir/-/pkg-dir-4.2.0.tgz", + "integrity": "sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ==", + "dependencies": { + "find-up": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/pkg-up": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/pkg-up/-/pkg-up-3.1.0.tgz", + "integrity": "sha512-nDywThFk1i4BQK4twPQ6TA4RT8bDY96yeuCVBWL3ePARCiEKDRSrNGbFIgUJpLp+XeIR65v8ra7WuJOFUBtkMA==", + "dependencies": { + "find-up": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/pkg-up/node_modules/find-up": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-3.0.0.tgz", + "integrity": "sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg==", + "dependencies": { + "locate-path": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/pkg-up/node_modules/locate-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-3.0.0.tgz", + "integrity": "sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A==", + "dependencies": { + "p-locate": "^3.0.0", + "path-exists": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/pkg-up/node_modules/p-locate": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-3.0.0.tgz", + "integrity": "sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ==", + "dependencies": { + "p-limit": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/pkg-up/node_modules/path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha1-zg6+ql94yxiSXqfYENe1mwEP1RU=", + "engines": { + "node": ">=4" + } + }, + "node_modules/platform": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/platform/-/platform-1.3.6.tgz", + "integrity": "sha512-fnWVljUchTro6RiCFvCXBbNhJc2NijN7oIQxbwsyL0buWJPG85v81ehlHI9fXrJsMNgTofEoWIQeClKpgxFLrg==" + }, + "node_modules/pngjs": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/pngjs/-/pngjs-3.4.0.tgz", + "integrity": "sha512-NCrCHhWmnQklfH4MtJMRjZ2a8c80qXeMlQMv2uVp9ISJMTt562SbGd6n2oq0PaPgKm7Z6pL9E2UlLIhC+SHL3w==", + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/postcss": { + "version": "8.4.13", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-8.4.13.tgz", + "integrity": "sha512-jtL6eTBrza5MPzy8oJLFuUscHDXTV5KcLlqAWHl5q5WYRfnNRGSmOZmOZ1T6Gy7A99mOZfqungmZMpMmCVJ8ZA==", + "dependencies": { + "nanoid": "^3.3.3", + "picocolors": "^1.0.0", + "source-map-js": "^1.0.2" + }, + "engines": { + "node": "^10 || ^12 || >=14" + } + }, + "node_modules/postcss-calc": { + "version": "8.2.4", + "resolved": "https://registry.npmjs.org/postcss-calc/-/postcss-calc-8.2.4.tgz", + "integrity": "sha512-SmWMSJmB8MRnnULldx0lQIyhSNvuDl9HfrZkaqqE/WHAhToYsAvDq+yAsA/kIyINDszOp3Rh0GFoNuH5Ypsm3Q==", + "dependencies": { + "postcss-selector-parser": "^6.0.9", + "postcss-value-parser": "^4.2.0" + }, + "peerDependencies": { + "postcss": "^8.2.2" + } + }, + "node_modules/postcss-colormin": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/postcss-colormin/-/postcss-colormin-5.3.0.tgz", + "integrity": "sha512-WdDO4gOFG2Z8n4P8TWBpshnL3JpmNmJwdnfP2gbk2qBA8PWwOYcmjmI/t3CmMeL72a7Hkd+x/Mg9O2/0rD54Pg==", + "dependencies": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0", + "colord": "^2.9.1", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-convert-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-convert-values/-/postcss-convert-values-5.1.0.tgz", + "integrity": "sha512-GkyPbZEYJiWtQB0KZ0X6qusqFHUepguBCNFi9t5JJc7I2OTXG7C0twbTLvCfaKOLl3rSXmpAwV7W5txd91V84g==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-discard-comments": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-discard-comments/-/postcss-discard-comments-5.1.1.tgz", + "integrity": "sha512-5JscyFmvkUxz/5/+TB3QTTT9Gi9jHkcn8dcmmuN68JQcv3aQg4y88yEHHhwFB52l/NkaJ43O0dbksGMAo49nfQ==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-discard-duplicates": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-duplicates/-/postcss-discard-duplicates-5.1.0.tgz", + "integrity": "sha512-zmX3IoSI2aoenxHV6C7plngHWWhUOV3sP1T8y2ifzxzbtnuhk1EdPwm0S1bIUNaJ2eNbWeGLEwzw8huPD67aQw==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-discard-empty": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-discard-empty/-/postcss-discard-empty-5.1.1.tgz", + "integrity": "sha512-zPz4WljiSuLWsI0ir4Mcnr4qQQ5e1Ukc3i7UfE2XcrwKK2LIPIqE5jxMRxO6GbI3cv//ztXDsXwEWT3BHOGh3A==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-discard-overridden": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-overridden/-/postcss-discard-overridden-5.1.0.tgz", + "integrity": "sha512-21nOL7RqWR1kasIVdKs8HNqQJhFxLsyRfAnUDm4Fe4t4mCWL9OJiHvlHPjcd8zc5Myu89b/7wZDnOSjFgeWRtw==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-flexbugs-fixes": { + "version": "5.0.2", + "resolved": "https://registry.npmjs.org/postcss-flexbugs-fixes/-/postcss-flexbugs-fixes-5.0.2.tgz", + "integrity": "sha512-18f9voByak7bTktR2QgDveglpn9DTbBWPUzSOe9g0N4WR/2eSt6Vrcbf0hmspvMI6YWGywz6B9f7jzpFNJJgnQ==", + "peerDependencies": { + "postcss": "^8.1.4" + } + }, + "node_modules/postcss-loader": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/postcss-loader/-/postcss-loader-5.3.0.tgz", + "integrity": "sha512-/+Z1RAmssdiSLgIZwnJHwBMnlABPgF7giYzTN2NOfr9D21IJZ4mQC1R2miwp80zno9M4zMD/umGI8cR+2EL5zw==", + "dependencies": { + "cosmiconfig": "^7.0.0", + "klona": "^2.0.4", + "semver": "^7.3.4" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "postcss": "^7.0.0 || ^8.0.1", + "webpack": "^5.0.0" + } + }, + "node_modules/postcss-merge-longhand": { + "version": "5.1.4", + "resolved": "https://registry.npmjs.org/postcss-merge-longhand/-/postcss-merge-longhand-5.1.4.tgz", + "integrity": "sha512-hbqRRqYfmXoGpzYKeW0/NCZhvNyQIlQeWVSao5iKWdyx7skLvCfQFGIUsP9NUs3dSbPac2IC4Go85/zG+7MlmA==", + "dependencies": { + "postcss-value-parser": "^4.2.0", + "stylehacks": "^5.1.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-merge-rules": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-merge-rules/-/postcss-merge-rules-5.1.1.tgz", + "integrity": "sha512-8wv8q2cXjEuCcgpIB1Xx1pIy8/rhMPIQqYKNzEdyx37m6gpq83mQQdCxgIkFgliyEnKvdwJf/C61vN4tQDq4Ww==", + "dependencies": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0", + "cssnano-utils": "^3.1.0", + "postcss-selector-parser": "^6.0.5" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-minify-font-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-minify-font-values/-/postcss-minify-font-values-5.1.0.tgz", + "integrity": "sha512-el3mYTgx13ZAPPirSVsHqFzl+BBBDrXvbySvPGFnQcTI4iNslrPaFq4muTkLZmKlGk4gyFAYUBMH30+HurREyA==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-minify-gradients": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-minify-gradients/-/postcss-minify-gradients-5.1.1.tgz", + "integrity": "sha512-VGvXMTpCEo4qHTNSa9A0a3D+dxGFZCYwR6Jokk+/3oB6flu2/PnPXAh2x7x52EkY5xlIHLm+Le8tJxe/7TNhzw==", + "dependencies": { + "colord": "^2.9.1", + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-minify-params": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/postcss-minify-params/-/postcss-minify-params-5.1.2.tgz", + "integrity": "sha512-aEP+p71S/urY48HWaRHasyx4WHQJyOYaKpQ6eXl8k0kxg66Wt/30VR6/woh8THgcpRbonJD5IeD+CzNhPi1L8g==", + "dependencies": { + "browserslist": "^4.16.6", + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-minify-selectors": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/postcss-minify-selectors/-/postcss-minify-selectors-5.2.0.tgz", + "integrity": "sha512-vYxvHkW+iULstA+ctVNx0VoRAR4THQQRkG77o0oa4/mBS0OzGvvzLIvHDv/nNEM0crzN2WIyFU5X7wZhaUK3RA==", + "dependencies": { + "postcss-selector-parser": "^6.0.5" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-modules-extract-imports": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-extract-imports/-/postcss-modules-extract-imports-3.0.0.tgz", + "integrity": "sha512-bdHleFnP3kZ4NYDhuGlVK+CMrQ/pqUm8bx/oGL93K6gVwiclvX5x0n76fYMKuIGKzlABOy13zsvqjb0f92TEXw==", + "engines": { + "node": "^10 || ^12 || >= 14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/postcss-modules-local-by-default": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.0.tgz", + "integrity": "sha512-sT7ihtmGSF9yhm6ggikHdV0hlziDTX7oFoXtuVWeDd3hHObNkcHRo9V3yg7vCAY7cONyxJC/XXCmmiHHcvX7bQ==", + "dependencies": { + "icss-utils": "^5.0.0", + "postcss-selector-parser": "^6.0.2", + "postcss-value-parser": "^4.1.0" + }, + "engines": { + "node": "^10 || ^12 || >= 14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/postcss-modules-scope": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz", + "integrity": "sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg==", + "dependencies": { + "postcss-selector-parser": "^6.0.4" + }, + "engines": { + "node": "^10 || ^12 || >= 14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/postcss-modules-values": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-values/-/postcss-modules-values-4.0.0.tgz", + "integrity": "sha512-RDxHkAiEGI78gS2ofyvCsu7iycRv7oqw5xMWn9iMoR0N/7mf9D50ecQqUo5BZ9Zh2vH4bCUR/ktCqbB9m8vJjQ==", + "dependencies": { + "icss-utils": "^5.0.0" + }, + "engines": { + "node": "^10 || ^12 || >= 14" + }, + "peerDependencies": { + "postcss": "^8.1.0" + } + }, + "node_modules/postcss-normalize-charset": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-charset/-/postcss-normalize-charset-5.1.0.tgz", + "integrity": "sha512-mSgUJ+pd/ldRGVx26p2wz9dNZ7ji6Pn8VWBajMXFf8jk7vUoSrZ2lt/wZR7DtlZYKesmZI680qjr2CeFF2fbUg==", + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-display-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-display-values/-/postcss-normalize-display-values-5.1.0.tgz", + "integrity": "sha512-WP4KIM4o2dazQXWmFaqMmcvsKmhdINFblgSeRgn8BJ6vxaMyaJkwAzpPpuvSIoG/rmX3M+IrRZEz2H0glrQNEA==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-positions": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-positions/-/postcss-normalize-positions-5.1.0.tgz", + "integrity": "sha512-8gmItgA4H5xiUxgN/3TVvXRoJxkAWLW6f/KKhdsH03atg0cB8ilXnrB5PpSshwVu/dD2ZsRFQcR1OEmSBDAgcQ==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-repeat-style": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-repeat-style/-/postcss-normalize-repeat-style-5.1.0.tgz", + "integrity": "sha512-IR3uBjc+7mcWGL6CtniKNQ4Rr5fTxwkaDHwMBDGGs1x9IVRkYIT/M4NelZWkAOBdV6v3Z9S46zqaKGlyzHSchw==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-string": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-string/-/postcss-normalize-string-5.1.0.tgz", + "integrity": "sha512-oYiIJOf4T9T1N4i+abeIc7Vgm/xPCGih4bZz5Nm0/ARVJ7K6xrDlLwvwqOydvyL3RHNf8qZk6vo3aatiw/go3w==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-timing-functions": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-timing-functions/-/postcss-normalize-timing-functions-5.1.0.tgz", + "integrity": "sha512-DOEkzJ4SAXv5xkHl0Wa9cZLF3WCBhF3o1SKVxKQAa+0pYKlueTpCgvkFAHfk+Y64ezX9+nITGrDZeVGgITJXjg==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-unicode": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-unicode/-/postcss-normalize-unicode-5.1.0.tgz", + "integrity": "sha512-J6M3MizAAZ2dOdSjy2caayJLQT8E8K9XjLce8AUQMwOrCvjCHv24aLC/Lps1R1ylOfol5VIDMaM/Lo9NGlk1SQ==", + "dependencies": { + "browserslist": "^4.16.6", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-url": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-url/-/postcss-normalize-url-5.1.0.tgz", + "integrity": "sha512-5upGeDO+PVthOxSmds43ZeMeZfKH+/DKgGRD7TElkkyS46JXAUhMzIKiCa7BabPeIy3AQcTkXwVVN7DbqsiCew==", + "dependencies": { + "normalize-url": "^6.0.1", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-normalize-whitespace": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-normalize-whitespace/-/postcss-normalize-whitespace-5.1.1.tgz", + "integrity": "sha512-83ZJ4t3NUDETIHTa3uEg6asWjSBYL5EdkVB0sDncx9ERzOKBVJIUeDO9RyA9Zwtig8El1d79HBp0JEi8wvGQnA==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-ordered-values": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-ordered-values/-/postcss-ordered-values-5.1.1.tgz", + "integrity": "sha512-7lxgXF0NaoMIgyihL/2boNAEZKiW0+HkMhdKMTD93CjW8TdCy2hSdj8lsAo+uwm7EDG16Da2Jdmtqpedl0cMfw==", + "dependencies": { + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-reduce-initial": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-reduce-initial/-/postcss-reduce-initial-5.1.0.tgz", + "integrity": "sha512-5OgTUviz0aeH6MtBjHfbr57tml13PuedK/Ecg8szzd4XRMbYxH4572JFG067z+FqBIf6Zp/d+0581glkvvWMFw==", + "dependencies": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-reduce-transforms": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-reduce-transforms/-/postcss-reduce-transforms-5.1.0.tgz", + "integrity": "sha512-2fbdbmgir5AvpW9RLtdONx1QoYG2/EtqpNQbFASDlixBbAYuTcJ0dECwlqNqH7VbaUnEnh8SrxOe2sRIn24XyQ==", + "dependencies": { + "postcss-value-parser": "^4.2.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-selector-parser": { + "version": "6.0.10", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-6.0.10.tgz", + "integrity": "sha512-IQ7TZdoaqbT+LCpShg46jnZVlhWD2w6iQYAcYXfHARZ7X1t/UGhhceQDs5X0cGqKvYlHNOuv7Oa1xmb0oQuA3w==", + "dependencies": { + "cssesc": "^3.0.0", + "util-deprecate": "^1.0.2" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/postcss-svgo": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-svgo/-/postcss-svgo-5.1.0.tgz", + "integrity": "sha512-D75KsH1zm5ZrHyxPakAxJWtkyXew5qwS70v56exwvw542d9CRtTo78K0WeFxZB4G7JXKKMbEZtZayTGdIky/eA==", + "dependencies": { + "postcss-value-parser": "^4.2.0", + "svgo": "^2.7.0" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-unique-selectors": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-unique-selectors/-/postcss-unique-selectors-5.1.1.tgz", + "integrity": "sha512-5JiODlELrz8L2HwxfPnhOWZYWDxVHWL83ufOv84NrcgipI7TaeRsatAhK4Tr2/ZiYldpK/wBvw5BD3qfaK96GA==", + "dependencies": { + "postcss-selector-parser": "^6.0.5" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/postcss-value-parser": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/postcss-value-parser/-/postcss-value-parser-4.2.0.tgz", + "integrity": "sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ==" + }, + "node_modules/prebuild-install": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/prebuild-install/-/prebuild-install-7.1.0.tgz", + "integrity": "sha512-CNcMgI1xBypOyGqjp3wOc8AAo1nMhZS3Cwd3iHIxOdAUbb+YxdNuM4Z5iIrZ8RLvOsf3F3bl7b7xGq6DjQoNYA==", + "dependencies": { + "detect-libc": "^2.0.0", + "expand-template": "^2.0.3", + "github-from-package": "0.0.0", + "minimist": "^1.2.3", + "mkdirp-classic": "^0.5.3", + "napi-build-utils": "^1.0.1", + "node-abi": "^3.3.0", + "npmlog": "^4.0.1", + "pump": "^3.0.0", + "rc": "^1.2.7", + "simple-get": "^4.0.0", + "tar-fs": "^2.0.0", + "tunnel-agent": "^0.6.0" + }, + "bin": { + "prebuild-install": "bin.js" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/prebuild-install/node_modules/detect-libc": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.1.tgz", + "integrity": "sha512-463v3ZeIrcWtdgIg6vI6XUncguvr2TnGl4SzDXinkt9mSLpBJKXT3mW6xT3VQdDN11+WVs29pgvivTc4Lp8v+w==", + "engines": { + "node": ">=8" + } + }, + "node_modules/prelude-ls": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", + "integrity": "sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g==", + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/prepend-http": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/prepend-http/-/prepend-http-2.0.0.tgz", + "integrity": "sha1-6SQ0v6XqjBn0HN/UAddBo8gZ2Jc=", + "engines": { + "node": ">=4" + } + }, + "node_modules/prettier": { + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.6.2.tgz", + "integrity": "sha512-PkUpF+qoXTqhOeWL9fu7As8LXsIUZ1WYaJiY/a7McAQzxjk82OF0tibkFXVCDImZtWxbvojFjerkiLb0/q8mew==", + "dev": true, + "bin": { + "prettier": "bin-prettier.js" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/pretty-bytes": { + "version": "5.6.0", + "resolved": "https://registry.npmjs.org/pretty-bytes/-/pretty-bytes-5.6.0.tgz", + "integrity": "sha512-FFw039TmrBqFK8ma/7OL3sDz/VytdtJr044/QUJtH0wK9lb9jLq9tJyIxUwtQJHwar2BqtiA4iCWSwo9JLkzFg==", + "engines": { + "node": ">=6" + } + }, + "node_modules/pretty-error": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/pretty-error/-/pretty-error-2.1.2.tgz", + "integrity": "sha512-EY5oDzmsX5wvuynAByrmY0P0hcp+QpnAKbJng2A2MPjVKXCxrDSUkzghVJ4ZGPIv+JC4gX8fPUWscC0RtjsWGw==", + "dependencies": { + "lodash": "^4.17.20", + "renderkid": "^2.0.4" + } + }, + "node_modules/probe-image-size": { + "version": "7.2.3", + "resolved": "https://registry.npmjs.org/probe-image-size/-/probe-image-size-7.2.3.tgz", + "integrity": "sha512-HubhG4Rb2UH8YtV4ba0Vp5bQ7L78RTONYu/ujmCu5nBI8wGv24s4E9xSKBi0N1MowRpxk76pFCpJtW0KPzOK0w==", + "dependencies": { + "lodash.merge": "^4.6.2", + "needle": "^2.5.2", + "stream-parser": "~0.3.1" + } + }, + "node_modules/process": { + "version": "0.11.10", + "resolved": "https://registry.npmjs.org/process/-/process-0.11.10.tgz", + "integrity": "sha1-czIwDoQBYb2j5podHZGn1LwW8YI=", + "engines": { + "node": ">= 0.6.0" + } + }, + "node_modules/process-nextick-args": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/process-nextick-args/-/process-nextick-args-2.0.1.tgz", + "integrity": "sha512-3ouUOpQhtgrbOa17J7+uxOTpITYWaGP7/AhoR3+A+/1e9skrzelGi/dXzEYyvbxubEF6Wn2ypscTKiKJFFn1ag==" + }, + "node_modules/progress": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/progress/-/progress-2.0.3.tgz", + "integrity": "sha512-7PiHtLll5LdnKIMw100I+8xJXR5gW2QwWYkT6iJva0bXitZKa/XMrSbdmg3r2Xnaidz9Qumd0VPaMrZlF9V9sA==", + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/promise": { + "version": "7.3.1", + "resolved": "https://registry.npmjs.org/promise/-/promise-7.3.1.tgz", + "integrity": "sha512-nolQXZ/4L+bP/UGlkfaIujX9BKxGwmQ9OT4mOt5yvy8iK1h3wqTEJCijzGANTCCl9nWjY41juyAn2K3Q1hLLTg==", + "dependencies": { + "asap": "~2.0.3" + } + }, + "node_modules/prompts": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/prompts/-/prompts-2.4.2.tgz", + "integrity": "sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q==", + "dependencies": { + "kleur": "^3.0.3", + "sisteransi": "^1.0.5" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/prop-types": { + "version": "15.8.1", + "resolved": "https://registry.npmjs.org/prop-types/-/prop-types-15.8.1.tgz", + "integrity": "sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg==", + "dependencies": { + "loose-envify": "^1.4.0", + "object-assign": "^4.1.1", + "react-is": "^16.13.1" + } + }, + "node_modules/proper-lockfile": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/proper-lockfile/-/proper-lockfile-4.1.2.tgz", + "integrity": "sha512-TjNPblN4BwAWMXU8s9AEz4JmQxnD1NNL7bNOY/AKUzyamc379FWASUhc/K1pL2noVb+XmZKLL68cjzLsiOAMaA==", + "dependencies": { + "graceful-fs": "^4.2.4", + "retry": "^0.12.0", + "signal-exit": "^3.0.2" + } + }, + "node_modules/property-information": { + "version": "5.6.0", + "resolved": "https://registry.npmmirror.com/property-information/-/property-information-5.6.0.tgz", + "integrity": "sha512-YUHSPk+A30YPv+0Qf8i9Mbfe/C0hdPXk1s1jPVToV8pk8BQtpw10ct89Eo7OWkutrwqvT0eicAxlOg3dOAu8JA==", + "dependencies": { + "xtend": "^4.0.0" + } + }, + "node_modules/protocols": { + "version": "1.4.8", + "resolved": "https://registry.npmjs.org/protocols/-/protocols-1.4.8.tgz", + "integrity": "sha512-IgjKyaUSjsROSO8/D49Ab7hP8mJgTYcqApOqdPhLoPxAplXmkp+zRvsrSQjFn5by0rhm4VH0GAUELIPpx7B1yg==" + }, + "node_modules/proxy-addr": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/proxy-addr/-/proxy-addr-2.0.7.tgz", + "integrity": "sha512-llQsMLSUDUPT44jdrU/O37qlnifitDP+ZwrmmZcoSKyLKvtZxpyV0n2/bD/N4tBAAZ/gJEdZU7KMraoK1+XYAg==", + "dependencies": { + "forwarded": "0.2.0", + "ipaddr.js": "1.9.1" + }, + "engines": { + "node": ">= 0.10" + } + }, + "node_modules/pseudomap": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/pseudomap/-/pseudomap-1.0.2.tgz", + "integrity": "sha1-8FKijacOYYkX7wqKw0wa5aaChrM=" + }, + "node_modules/psl": { + "version": "1.8.0", + "resolved": "https://registry.npmmirror.com/psl/-/psl-1.8.0.tgz", + "integrity": "sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ==" + }, + "node_modules/pump": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/pump/-/pump-3.0.0.tgz", + "integrity": "sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww==", + "dependencies": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "node_modules/punycode": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.1.1.tgz", + "integrity": "sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A==", + "engines": { + "node": ">=6" + } + }, + "node_modules/pupa": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/pupa/-/pupa-2.1.1.tgz", + "integrity": "sha512-l1jNAspIBSFqbT+y+5FosojNpVpF94nlI+wDUpqP9enwOTfHx9f0gh5nB96vl+6yTpsJsypeNrwfzPrKuHB41A==", + "dependencies": { + "escape-goat": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/q": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/q/-/q-1.5.1.tgz", + "integrity": "sha1-fjL3W0E4EpHQRhHxvxQQmsAGUdc=", + "engines": { + "node": ">=0.6.0", + "teleport": ">=0.2.0" + } + }, + "node_modules/qs": { + "version": "6.10.3", + "resolved": "https://registry.npmjs.org/qs/-/qs-6.10.3.tgz", + "integrity": "sha512-wr7M2E0OFRfIfJZjKGieI8lBKb7fRCH4Fv5KNPEs7gJ8jadvotdsS08PzOKR7opXhZ/Xkjtt3WF9g38drmyRqQ==", + "dependencies": { + "side-channel": "^1.0.4" + }, + "engines": { + "node": ">=0.6" + } + }, + "node_modules/query-string": { + "version": "6.14.1", + "resolved": "https://registry.npmjs.org/query-string/-/query-string-6.14.1.tgz", + "integrity": "sha512-XDxAeVmpfu1/6IjyT/gXHOl+S0vQ9owggJ30hhWKdHAsNPOcasn5o9BW0eejZqL2e4vMjhAxoW3jVHcD6mbcYw==", + "dependencies": { + "decode-uri-component": "^0.2.0", + "filter-obj": "^1.1.0", + "split-on-first": "^1.0.0", + "strict-uri-encode": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/querystring": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/querystring/-/querystring-0.2.1.tgz", + "integrity": "sha512-wkvS7mL/JMugcup3/rMitHmd9ecIGd2lhFhK9N3UUQ450h66d1r3Y9nvXzQAW1Lq+wyx61k/1pfKS5KuKiyEbg==", + "deprecated": "The querystring API is considered Legacy. new code should use the URLSearchParams API instead.", + "engines": { + "node": ">=0.4.x" + } + }, + "node_modules/queue-microtask": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/queue-microtask/-/queue-microtask-1.2.3.tgz", + "integrity": "sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==" + }, + "node_modules/quick-lru": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/quick-lru/-/quick-lru-5.1.1.tgz", + "integrity": "sha512-WuyALRjWPDGtt/wzJiadO5AXY+8hZ80hVpe6MyivgraREW751X3SbhRvG3eLKOYN+8VEvqLcf3wdnt44Z4S4SA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/randombytes": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", + "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", + "dependencies": { + "safe-buffer": "^5.1.0" + } + }, + "node_modules/range-parser": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/range-parser/-/range-parser-1.2.1.tgz", + "integrity": "sha512-Hrgsx+orqoygnmhFbKaHE6c296J+HTAQXoxEF6gNupROmmGJRoyzfG3ccAveqCBrwr/2yxQ5BVd/GTl5agOwSg==", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/raw-body": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/raw-body/-/raw-body-2.5.1.tgz", + "integrity": "sha512-qqJBtEyVgS0ZmPGdCFPWJ3FreoqvG4MVQln/kCgF7Olq95IbOp0/BWyMwbdtn4VTvkM8Y7khCQ2Xgk/tcrCXig==", + "dependencies": { + "bytes": "3.1.2", + "http-errors": "2.0.0", + "iconv-lite": "0.4.24", + "unpipe": "1.0.0" + }, + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/raw-loader": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/raw-loader/-/raw-loader-4.0.2.tgz", + "integrity": "sha512-ZnScIV3ag9A4wPX/ZayxL/jZH+euYb6FcUinPcgiQW0+UBtEv0O6Q3lGd3cqJ+GHH+rksEv3Pj99oxJ3u3VIKA==", + "dependencies": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/raw-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/rc": { + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/rc/-/rc-1.2.8.tgz", + "integrity": "sha512-y3bGgqKj3QBdxLbLkomlohkvsA8gdAiUQlSBJnBhfn+BPxg4bc62d8TcBW15wavDfgexCgccckhcZvywyQYPOw==", + "dependencies": { + "deep-extend": "^0.6.0", + "ini": "~1.3.0", + "minimist": "^1.2.0", + "strip-json-comments": "~2.0.1" + }, + "bin": { + "rc": "cli.js" + } + }, + "node_modules/rc/node_modules/ini": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/ini/-/ini-1.3.8.tgz", + "integrity": "sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==" + }, + "node_modules/rc/node_modules/strip-json-comments": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-2.0.1.tgz", + "integrity": "sha1-PFMZQukIwml8DsNEhYwobHygpgo=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/react": { + "version": "17.0.1", + "integrity": "sha512-lG9c9UuMHdcAexXtigOZLX8exLWkW0Ku29qPRU8uhF2R9BN96dLCt0psvzPLlHc5OWkgymP3qwTRgbnw5BKx3w==", + "dependencies": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/react-dev-utils": { + "version": "12.0.1", + "resolved": "https://registry.npmjs.org/react-dev-utils/-/react-dev-utils-12.0.1.tgz", + "integrity": "sha512-84Ivxmr17KjUupyqzFode6xKhjwuEJDROWKJy/BthkL7Wn6NJ8h4WE6k/exAv6ImS+0oZLRRW5j/aINMHyeGeQ==", + "dependencies": { + "@babel/code-frame": "^7.16.0", + "address": "^1.1.2", + "browserslist": "^4.18.1", + "chalk": "^4.1.2", + "cross-spawn": "^7.0.3", + "detect-port-alt": "^1.1.6", + "escape-string-regexp": "^4.0.0", + "filesize": "^8.0.6", + "find-up": "^5.0.0", + "fork-ts-checker-webpack-plugin": "^6.5.0", + "global-modules": "^2.0.0", + "globby": "^11.0.4", + "gzip-size": "^6.0.0", + "immer": "^9.0.7", + "is-root": "^2.1.0", + "loader-utils": "^3.2.0", + "open": "^8.4.0", + "pkg-up": "^3.1.0", + "prompts": "^2.4.2", + "react-error-overlay": "^6.0.11", + "recursive-readdir": "^2.2.2", + "shell-quote": "^1.7.3", + "strip-ansi": "^6.0.1", + "text-table": "^0.2.0" + }, + "engines": { + "node": ">=14" + } + }, + "node_modules/react-dev-utils/node_modules/cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "dependencies": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/react-dev-utils/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/react-dev-utils/node_modules/detect-port-alt": { + "version": "1.1.6", + "resolved": "https://registry.npmjs.org/detect-port-alt/-/detect-port-alt-1.1.6.tgz", + "integrity": "sha512-5tQykt+LqfJFBEYaDITx7S7cR7mJ/zQmLXZ2qt5w04ainYZw6tBf9dBunMjVeVOdYVRUzUOE4HkY5J7+uttb5Q==", + "dependencies": { + "address": "^1.0.1", + "debug": "^2.6.0" + }, + "bin": { + "detect": "bin/detect-port", + "detect-port": "bin/detect-port" + }, + "engines": { + "node": ">= 4.2.1" + } + }, + "node_modules/react-dev-utils/node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "engines": { + "node": ">=10" + } + }, + "node_modules/react-dev-utils/node_modules/find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "dependencies": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/react-dev-utils/node_modules/loader-utils": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-3.2.0.tgz", + "integrity": "sha512-HVl9ZqccQihZ7JM85dco1MvO9G+ONvxoGa9rkhzFsneGLKSUg1gJf9bWzhRhcvm2qChhWpebQhP44qxjKIUCaQ==", + "engines": { + "node": ">= 12.13.0" + } + }, + "node_modules/react-dev-utils/node_modules/locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "dependencies": { + "p-locate": "^5.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/react-dev-utils/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/react-dev-utils/node_modules/open": { + "version": "8.4.0", + "resolved": "https://registry.npmjs.org/open/-/open-8.4.0.tgz", + "integrity": "sha512-XgFPPM+B28FtCCgSb9I+s9szOC1vZRSwgWsRUA5ylIxRTgKozqjOCrVOqGsYABPYK5qnfqClxZTFBa8PKt2v6Q==", + "dependencies": { + "define-lazy-prop": "^2.0.0", + "is-docker": "^2.1.1", + "is-wsl": "^2.2.0" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/react-dev-utils/node_modules/p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dependencies": { + "yocto-queue": "^0.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/react-dev-utils/node_modules/p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "dependencies": { + "p-limit": "^3.0.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/react-dev-utils/node_modules/path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/react-dev-utils/node_modules/shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "dependencies": { + "shebang-regex": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/react-dev-utils/node_modules/shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", + "engines": { + "node": ">=8" + } + }, + "node_modules/react-dev-utils/node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/react-dom": { + "version": "17.0.1", + "integrity": "sha512-6eV150oJZ9U2t9svnsspTMrWNyHc6chX0KzDeAOXftRa8bNeOKTTfCJ7KorIwenkHd2xqVTBTCZd79yk/lx/Ug==", + "dependencies": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1", + "scheduler": "^0.20.1" + }, + "peerDependencies": { + "react": "17.0.1" + } + }, + "node_modules/react-error-overlay": { + "version": "6.0.11", + "resolved": "https://registry.npmjs.org/react-error-overlay/-/react-error-overlay-6.0.11.tgz", + "integrity": "sha512-/6UZ2qgEyH2aqzYZgQPxEnz33NJ2gNsnHA2o5+o4wW9bLM/JYQitNP9xPhsXwC08hMMovfGe/8retsdDsczPRg==" + }, + "node_modules/react-fast-compare": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/react-fast-compare/-/react-fast-compare-3.2.0.tgz", + "integrity": "sha512-rtGImPZ0YyLrscKI9xTpV8psd6I8VAtjKCzQDlzyDvqJA8XOW78TXYQwNRNd8g8JZnDu8q9Fu/1v4HPAVwVdHA==" + }, + "node_modules/react-helmet": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/react-helmet/-/react-helmet-6.1.0.tgz", + "integrity": "sha512-4uMzEY9nlDlgxr61NL3XbKRy1hEkXmKNXhjbAIOVw5vcFrsdYbH2FEwcNyWvWinl103nXgzYNlns9ca+8kFiWw==", + "dependencies": { + "object-assign": "^4.1.1", + "prop-types": "^15.7.2", + "react-fast-compare": "^3.1.1", + "react-side-effect": "^2.1.0" + }, + "peerDependencies": { + "react": ">=16.3.0" + } + }, + "node_modules/react-is": { + "version": "16.13.1", + "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.13.1.tgz", + "integrity": "sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==" + }, + "node_modules/react-lifecycles-compat": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz", + "integrity": "sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA==" + }, + "node_modules/react-popper": { + "version": "2.3.0", + "resolved": "https://registry.npmmirror.com/react-popper/-/react-popper-2.3.0.tgz", + "integrity": "sha512-e1hj8lL3uM+sgSR4Lxzn5h1GxBlpa4CQz0XLF8kx4MDrDRWY0Ena4c97PUeSX9i5W3UAfDP0z0FXCTQkoXUl3Q==", + "dependencies": { + "react-fast-compare": "^3.0.1", + "warning": "^4.0.2" + }, + "peerDependencies": { + "@popperjs/core": "^2.0.0", + "react": "^16.8.0 || ^17 || ^18", + "react-dom": "^16.8.0 || ^17 || ^18" + } + }, + "node_modules/react-property": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/react-property/-/react-property-2.0.0.tgz", + "integrity": "sha512-kzmNjIgU32mO4mmH5+iUyrqlpFQhF8K2k7eZ4fdLSOPFrD1XgEuSBv9LDEgxRXTMBqMd8ppT0x6TIzqE5pdGdw==" + }, + "node_modules/react-refresh": { + "version": "0.9.0", + "resolved": "https://registry.npmjs.org/react-refresh/-/react-refresh-0.9.0.tgz", + "integrity": "sha512-Gvzk7OZpiqKSkxsQvO/mbTN1poglhmAV7gR/DdIrRrSMXraRQQlfikRJOr3Nb9GTMPC5kof948Zy6jJZIFtDvQ==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/react-side-effect": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/react-side-effect/-/react-side-effect-2.1.1.tgz", + "integrity": "sha512-2FoTQzRNTncBVtnzxFOk2mCpcfxQpenBMbk5kSVBg5UcPqV9fRbgY2zhb7GTWWOlpFmAxhClBDlIq8Rsubz1yQ==", + "peerDependencies": { + "react": "^16.3.0 || ^17.0.0" + } + }, + "node_modules/react-textarea-autosize": { + "version": "8.3.3", + "resolved": "https://registry.npmmirror.com/react-textarea-autosize/-/react-textarea-autosize-8.3.3.tgz", + "integrity": "sha512-2XlHXK2TDxS6vbQaoPbMOfQ8GK7+irc2fVK6QFIcC8GOnH3zI/v481n+j1L0WaPVvKxwesnY93fEfH++sus2rQ==", + "dependencies": { + "@babel/runtime": "^7.10.2", + "use-composed-ref": "^1.0.0", + "use-latest": "^1.0.0" + }, + "engines": { + "node": ">=10" + }, + "peerDependencies": { + "react": "^16.8.0 || ^17.0.0" + } + }, + "node_modules/read": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/read/-/read-1.0.7.tgz", + "integrity": "sha1-s9oZvQUkMal2cdRKQmNK33ELQMQ=", + "dependencies": { + "mute-stream": "~0.0.4" + }, + "engines": { + "node": ">=0.8" + } + }, + "node_modules/read-pkg": { + "version": "5.2.0", + "resolved": "https://registry.npmmirror.com/read-pkg/-/read-pkg-5.2.0.tgz", + "integrity": "sha512-Ug69mNOpfvKDAc2Q8DRpMjjzdtrnv9HcSMX+4VsZxD1aZ6ZzrIE7rlzXBtWTyhULSMKg076AW6WR5iZpD0JiOg==", + "dependencies": { + "@types/normalize-package-data": "^2.4.0", + "normalize-package-data": "^2.5.0", + "parse-json": "^5.0.0", + "type-fest": "^0.6.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/read-pkg-up": { + "version": "7.0.1", + "resolved": "https://registry.npmmirror.com/read-pkg-up/-/read-pkg-up-7.0.1.tgz", + "integrity": "sha512-zK0TB7Xd6JpCLmlLmufqykGE+/TlOePD6qKClNW7hHDKFh/J7/7gCWGR7joEQEW1bKq3a3yUZSObOoWLFQ4ohg==", + "dependencies": { + "find-up": "^4.1.0", + "read-pkg": "^5.2.0", + "type-fest": "^0.8.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/read-pkg-up/node_modules/type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/read-pkg/node_modules/hosted-git-info": { + "version": "2.8.9", + "resolved": "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz", + "integrity": "sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw==" + }, + "node_modules/read-pkg/node_modules/normalize-package-data": { + "version": "2.5.0", + "resolved": "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-2.5.0.tgz", + "integrity": "sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA==", + "dependencies": { + "hosted-git-info": "^2.1.4", + "resolve": "^1.10.0", + "semver": "2 || 3 || 4 || 5", + "validate-npm-package-license": "^3.0.1" + } + }, + "node_modules/read-pkg/node_modules/semver": { + "version": "5.7.1", + "resolved": "https://registry.npmmirror.com/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==", + "bin": { + "semver": "bin/semver" + } + }, + "node_modules/read-pkg/node_modules/type-fest": { + "version": "0.6.0", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.6.0.tgz", + "integrity": "sha512-q+MB8nYR1KDLrgr4G5yemftpMC7/QLqVndBmEEdqzmNj5dcFOO4Oo8qlwZE3ULT3+Zim1F8Kq4cBnikNhlCMlg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/readable-stream": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.0.tgz", + "integrity": "sha512-BViHy7LKeTz4oNnkcLJ+lVSL6vpiFeX6/d3oSH8zCW7UxP2onchk+vTGB143xuFjHS3deTgkKoXXymXqymiIdA==", + "dependencies": { + "inherits": "^2.0.3", + "string_decoder": "^1.1.1", + "util-deprecate": "^1.0.1" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/readable-web-to-node-stream": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/readable-web-to-node-stream/-/readable-web-to-node-stream-3.0.2.tgz", + "integrity": "sha512-ePeK6cc1EcKLEhJFt/AebMCLL+GgSKhuygrZ/GLaKZYEecIgIECf4UaUuaByiGtzckwR4ain9VzUh95T1exYGw==", + "dependencies": { + "readable-stream": "^3.6.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/readdirp": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz", + "integrity": "sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==", + "dependencies": { + "picomatch": "^2.2.1" + }, + "engines": { + "node": ">=8.10.0" + } + }, + "node_modules/recursive-readdir": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/recursive-readdir/-/recursive-readdir-2.2.2.tgz", + "integrity": "sha512-nRCcW9Sj7NuZwa2XvH9co8NPeXUBhZP7CRKJtU+cS6PW9FpCIFoI5ib0NT1ZrbNuPoRy0ylyCaUL8Gih4LSyFg==", + "dependencies": { + "minimatch": "3.0.4" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/recursive-readdir/node_modules/minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, + "node_modules/redent": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/redent/-/redent-3.0.0.tgz", + "integrity": "sha512-6tDA8g98We0zd0GvVeMT9arEOnTw9qM03L9cJXaCjrip1OO764RDBLBfrB4cwzNGDj5OA5ioymC9GkizgWJDUg==", + "dependencies": { + "indent-string": "^4.0.0", + "strip-indent": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/redux": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/redux/-/redux-4.1.2.tgz", + "integrity": "sha512-SH8PglcebESbd/shgf6mii6EIoRM0zrQyjcuQ+ojmfxjTtE0z9Y8pa62iA/OJ58qjP6j27uyW4kUF4jl/jd6sw==", + "dependencies": { + "@babel/runtime": "^7.9.2" + } + }, + "node_modules/redux-thunk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/redux-thunk/-/redux-thunk-2.4.1.tgz", + "integrity": "sha512-OOYGNY5Jy2TWvTL1KgAlVy6dcx3siPJ1wTq741EPyUKfn6W6nChdICjZwCd0p8AZBs5kWpZlbkXW2nE/zjUa+Q==", + "peerDependencies": { + "redux": "^4" + } + }, + "node_modules/regenerate": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/regenerate/-/regenerate-1.4.2.tgz", + "integrity": "sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A==" + }, + "node_modules/regenerate-unicode-properties": { + "version": "10.0.1", + "resolved": "https://registry.npmjs.org/regenerate-unicode-properties/-/regenerate-unicode-properties-10.0.1.tgz", + "integrity": "sha512-vn5DU6yg6h8hP/2OkQo3K7uVILvY4iu0oI4t3HFa81UPkhGJwkRwM10JEc3upjdhHjs/k8GJY1sRBhk5sr69Bw==", + "dependencies": { + "regenerate": "^1.4.2" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/regenerator-runtime": { + "version": "0.13.9", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz", + "integrity": "sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA==" + }, + "node_modules/regenerator-transform": { + "version": "0.15.0", + "resolved": "https://registry.npmjs.org/regenerator-transform/-/regenerator-transform-0.15.0.tgz", + "integrity": "sha512-LsrGtPmbYg19bcPHwdtmXwbW+TqNvtY4riE3P83foeHRroMbH6/2ddFBfab3t7kbzc7v7p4wbkIecHImqt0QNg==", + "dependencies": { + "@babel/runtime": "^7.8.4" + } + }, + "node_modules/regex-parser": { + "version": "2.2.11", + "resolved": "https://registry.npmmirror.com/regex-parser/-/regex-parser-2.2.11.tgz", + "integrity": "sha512-jbD/FT0+9MBU2XAZluI7w2OBs1RBi6p9M83nkoZayQXXU9e8Robt69FcZc7wU4eJD/YFTjn1JdCk3rbMJajz8Q==" + }, + "node_modules/regexp.prototype.flags": { + "version": "1.4.3", + "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.4.3.tgz", + "integrity": "sha512-fjggEOO3slI6Wvgjwflkc4NFRCTZAu5CnNfBd5qOMYhWdn67nJBBu34/TkD++eeFmd8C9r9jfXJ27+nSiRkSUA==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "functions-have-names": "^1.2.2" + }, + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/regexpp": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/regexpp/-/regexpp-3.2.0.tgz", + "integrity": "sha512-pq2bWo9mVD43nbts2wGv17XLiNLya+GklZ8kaDLV2Z08gDCsGpnKn9BFMepvWuHCbyVvY7J5o5+BVvoQbmlJLg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/regexpu-core": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/regexpu-core/-/regexpu-core-5.0.1.tgz", + "integrity": "sha512-CriEZlrKK9VJw/xQGJpQM5rY88BtuL8DM+AEwvcThHilbxiTAy8vq4iJnd2tqq8wLmjbGZzP7ZcKFjbGkmEFrw==", + "dependencies": { + "regenerate": "^1.4.2", + "regenerate-unicode-properties": "^10.0.1", + "regjsgen": "^0.6.0", + "regjsparser": "^0.8.2", + "unicode-match-property-ecmascript": "^2.0.0", + "unicode-match-property-value-ecmascript": "^2.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/registry-auth-token": { + "version": "4.2.1", + "resolved": "https://registry.npmjs.org/registry-auth-token/-/registry-auth-token-4.2.1.tgz", + "integrity": "sha512-6gkSb4U6aWJB4SF2ZvLb76yCBjcvufXBqvvEx1HbmKPkutswjW1xNVRY0+daljIYRbogN7O0etYSlbiaEQyMyw==", + "dependencies": { + "rc": "^1.2.8" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/registry-url": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/registry-url/-/registry-url-5.1.0.tgz", + "integrity": "sha512-8acYXXTI0AkQv6RAOjE3vOaIXZkT9wo4LOFbBKYQEEnnMNBpKqdUrI6S4NT0KPIo/WVvJ5tE/X5LF/TQUf0ekw==", + "dependencies": { + "rc": "^1.2.8" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/regjsgen": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/regjsgen/-/regjsgen-0.6.0.tgz", + "integrity": "sha512-ozE883Uigtqj3bx7OhL1KNbCzGyW2NQZPl6Hs09WTvCuZD5sTI4JY58bkbQWa/Y9hxIsvJ3M8Nbf7j54IqeZbA==" + }, + "node_modules/regjsparser": { + "version": "0.8.4", + "resolved": "https://registry.npmjs.org/regjsparser/-/regjsparser-0.8.4.tgz", + "integrity": "sha512-J3LABycON/VNEu3abOviqGHuB/LOtOQj8SKmfP9anY5GfAVw/SPjwzSjxGjbZXIxbGfqTHtJw58C2Li/WkStmA==", + "dependencies": { + "jsesc": "~0.5.0" + }, + "bin": { + "regjsparser": "bin/parser" + } + }, + "node_modules/regjsparser/node_modules/jsesc": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-0.5.0.tgz", + "integrity": "sha1-597mbjXW/Bb3EP6R1c9p9w8IkR0=", + "bin": { + "jsesc": "bin/jsesc" + } + }, + "node_modules/relay-compiler": { + "version": "12.0.0", + "resolved": "https://registry.npmjs.org/relay-compiler/-/relay-compiler-12.0.0.tgz", + "integrity": "sha512-SWqeSQZ+AMU/Cr7iZsHi1e78Z7oh00I5SvR092iCJq79aupqJ6Ds+I1Pz/Vzo5uY5PY0jvC4rBJXzlIN5g9boQ==", + "dependencies": { + "@babel/core": "^7.14.0", + "@babel/generator": "^7.14.0", + "@babel/parser": "^7.14.0", + "@babel/runtime": "^7.0.0", + "@babel/traverse": "^7.14.0", + "@babel/types": "^7.0.0", + "babel-preset-fbjs": "^3.4.0", + "chalk": "^4.0.0", + "fb-watchman": "^2.0.0", + "fbjs": "^3.0.0", + "glob": "^7.1.1", + "immutable": "~3.7.6", + "invariant": "^2.2.4", + "nullthrows": "^1.1.1", + "relay-runtime": "12.0.0", + "signedsource": "^1.0.0", + "yargs": "^15.3.1" + }, + "bin": { + "relay-compiler": "bin/relay-compiler" + }, + "peerDependencies": { + "graphql": "^15.0.0" + } + }, + "node_modules/relay-runtime": { + "version": "12.0.0", + "resolved": "https://registry.npmjs.org/relay-runtime/-/relay-runtime-12.0.0.tgz", + "integrity": "sha512-QU6JKr1tMsry22DXNy9Whsq5rmvwr3LSZiiWV/9+DFpuTWvp+WFhobWMc8TC4OjKFfNhEZy7mOiqUAn5atQtug==", + "dependencies": { + "@babel/runtime": "^7.0.0", + "fbjs": "^3.0.0", + "invariant": "^2.2.4" + } + }, + "node_modules/remark": { + "version": "13.0.0", + "resolved": "https://registry.npmmirror.com/remark/-/remark-13.0.0.tgz", + "integrity": "sha512-HDz1+IKGtOyWN+QgBiAT0kn+2s6ovOxHyPAFGKVE81VSzJ+mq7RwHFledEvB5F1p4iJvOah/LOKdFuzvRnNLCA==", + "dependencies": { + "remark-parse": "^9.0.0", + "remark-stringify": "^9.0.0", + "unified": "^9.1.0" + } + }, + "node_modules/remark-footnotes": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/remark-footnotes/-/remark-footnotes-3.0.0.tgz", + "integrity": "sha512-ZssAvH9FjGYlJ/PBVKdSmfyPc3Cz4rTWgZLI4iE/SX8Nt5l3o3oEjv3wwG5VD7xOjktzdwp5coac+kJV9l4jgg==", + "dependencies": { + "mdast-util-footnote": "^0.1.0", + "micromark-extension-footnote": "^0.3.0" + } + }, + "node_modules/remark-gfm": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/remark-gfm/-/remark-gfm-1.0.0.tgz", + "integrity": "sha512-KfexHJCiqvrdBZVbQ6RopMZGwaXz6wFJEfByIuEwGf0arvITHjiKKZ1dpXujjH9KZdm1//XJQwgfnJ3lmXaDPA==", + "dependencies": { + "mdast-util-gfm": "^0.1.0", + "micromark-extension-gfm": "^0.3.0" + } + }, + "node_modules/remark-parse": { + "version": "9.0.0", + "resolved": "https://registry.npmmirror.com/remark-parse/-/remark-parse-9.0.0.tgz", + "integrity": "sha512-geKatMwSzEXKHuzBNU1z676sGcDcFoChMK38TgdHJNAYfFtsfHDQG7MoJAjs6sgYMqyLduCYWDIWZIxiPeafEw==", + "dependencies": { + "mdast-util-from-markdown": "^0.8.0" + } + }, + "node_modules/remark-retext": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/remark-retext/-/remark-retext-4.0.0.tgz", + "integrity": "sha512-cYCchalpf25bTtfXF24ribYvqytPKq0TiEhqQDBHvVEEsApebwruPWP1cTcvTFBidmpXyqzycm+y8ng7Kmvc8Q==", + "dependencies": { + "mdast-util-to-nlcst": "^4.0.0" + } + }, + "node_modules/remark-stringify": { + "version": "9.0.1", + "resolved": "https://registry.npmmirror.com/remark-stringify/-/remark-stringify-9.0.1.tgz", + "integrity": "sha512-mWmNg3ZtESvZS8fv5PTvaPckdL4iNlCHTt8/e/8oN08nArHRHjNZMKzA/YW3+p7/lYqIw4nx1XsjCBo/AxNChg==", + "dependencies": { + "mdast-util-to-markdown": "^0.6.0" + } + }, + "node_modules/remove-trailing-separator": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/remove-trailing-separator/-/remove-trailing-separator-1.1.0.tgz", + "integrity": "sha1-wkvOKig62tW8P1jg1IJJuSN52O8=" + }, + "node_modules/renderkid": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/renderkid/-/renderkid-2.0.7.tgz", + "integrity": "sha512-oCcFyxaMrKsKcTY59qnCAtmDVSLfPbrv6A3tVbPdFMMrv5jaK10V6m40cKsoPNhAqN6rmHW9sswW4o3ruSrwUQ==", + "dependencies": { + "css-select": "^4.1.3", + "dom-converter": "^0.2.0", + "htmlparser2": "^6.1.0", + "lodash": "^4.17.21", + "strip-ansi": "^3.0.1" + } + }, + "node_modules/renderkid/node_modules/ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/renderkid/node_modules/strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", + "dependencies": { + "ansi-regex": "^2.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/repeat-string": { + "version": "1.6.1", + "resolved": "https://registry.npmmirror.com/repeat-string/-/repeat-string-1.6.1.tgz", + "integrity": "sha512-PV0dzCYDNfRi1jCDbJzpW7jNNDRuCOG/jI5ctQcGKt/clZD+YcPS3yIlWuTJMmESC8aevCFmWJy5wjAFgNqN6w==", + "engines": { + "node": ">=0.10" + } + }, + "node_modules/request": { + "version": "2.88.2", + "resolved": "https://registry.npmmirror.com/request/-/request-2.88.2.tgz", + "integrity": "sha512-MsvtOrfG9ZcrOwAW+Qi+F6HbD0CWXEh9ou77uOb7FM2WPhwT7smM833PzanhJLsgXjN89Ir6V2PczXNnMpwKhw==", + "deprecated": "request has been deprecated, see https://github.com/request/request/issues/3142", + "dependencies": { + "aws-sign2": "~0.7.0", + "aws4": "^1.8.0", + "caseless": "~0.12.0", + "combined-stream": "~1.0.6", + "extend": "~3.0.2", + "forever-agent": "~0.6.1", + "form-data": "~2.3.2", + "har-validator": "~5.1.3", + "http-signature": "~1.2.0", + "is-typedarray": "~1.0.0", + "isstream": "~0.1.2", + "json-stringify-safe": "~5.0.1", + "mime-types": "~2.1.19", + "oauth-sign": "~0.9.0", + "performance-now": "^2.1.0", + "qs": "~6.5.2", + "safe-buffer": "^5.1.2", + "tough-cookie": "~2.5.0", + "tunnel-agent": "^0.6.0", + "uuid": "^3.3.2" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/request/node_modules/form-data": { + "version": "2.3.3", + "resolved": "https://registry.npmmirror.com/form-data/-/form-data-2.3.3.tgz", + "integrity": "sha512-1lLKB2Mu3aGP1Q/2eCOx0fNbRMe7XdwktwOruhfqqd0rIJWwN4Dh+E3hrPSlDCXnSR7UtZ1N38rVXm+6+MEhJQ==", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.6", + "mime-types": "^2.1.12" + }, + "engines": { + "node": ">= 0.12" + } + }, + "node_modules/request/node_modules/qs": { + "version": "6.5.3", + "resolved": "https://registry.npmmirror.com/qs/-/qs-6.5.3.tgz", + "integrity": "sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA==", + "engines": { + "node": ">=0.6" + } + }, + "node_modules/request/node_modules/uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmmirror.com/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==", + "deprecated": "Please upgrade to version 7 or higher. Older versions may use Math.random() in certain circumstances, which is known to be problematic. See https://v8.dev/blog/math-random for details.", + "bin": { + "uuid": "bin/uuid" + } + }, + "node_modules/require-directory": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", + "integrity": "sha1-jGStX9MNqxyXbiNE/+f3kqam30I=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/require-from-string": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/require-from-string/-/require-from-string-2.0.2.tgz", + "integrity": "sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/require-main-filename": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-2.0.0.tgz", + "integrity": "sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg==" + }, + "node_modules/require-package-name": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/require-package-name/-/require-package-name-2.0.1.tgz", + "integrity": "sha1-wR6XJ2tluOKSP3Xav1+y7ww4Qbk=" + }, + "node_modules/requires-port": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/requires-port/-/requires-port-1.0.0.tgz", + "integrity": "sha1-kl0mAdOaxIXgkc8NpcbmlNw9yv8=" + }, + "node_modules/resolve": { + "version": "1.22.0", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.22.0.tgz", + "integrity": "sha512-Hhtrw0nLeSrFQ7phPp4OOcVjLPIeMnRlr5mcnVuMe7M/7eBn98A3hmFRLoFo3DLZkivSYwhRUJTyPyWAk56WLw==", + "dependencies": { + "is-core-module": "^2.8.1", + "path-parse": "^1.0.7", + "supports-preserve-symlinks-flag": "^1.0.0" + }, + "bin": { + "resolve": "bin/resolve" + } + }, + "node_modules/resolve-alpn": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/resolve-alpn/-/resolve-alpn-1.2.1.tgz", + "integrity": "sha512-0a1F4l73/ZFZOakJnQ3FvkJ2+gSTQWz/r2KE5OdDY0TxPm5h4GkqkWWfM47T7HsbnOtcJVEF4epCVy6u7Q3K+g==" + }, + "node_modules/resolve-cwd": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/resolve-cwd/-/resolve-cwd-3.0.0.tgz", + "integrity": "sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg==", + "dependencies": { + "resolve-from": "^5.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/resolve-from": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-5.0.0.tgz", + "integrity": "sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/resolve-url": { + "version": "0.2.1", + "resolved": "https://registry.npmmirror.com/resolve-url/-/resolve-url-0.2.1.tgz", + "integrity": "sha512-ZuF55hVUQaaczgOIwqWzkEcEidmlD/xl44x1UZnhOXcYuFN2S6+rcxpG+C1N3So0wvNI3DmJICUFfu2SxhBmvg==", + "deprecated": "https://github.com/lydell/resolve-url#deprecated" + }, + "node_modules/resolve-url-loader": { + "version": "3.1.4", + "resolved": "https://registry.npmmirror.com/resolve-url-loader/-/resolve-url-loader-3.1.4.tgz", + "integrity": "sha512-D3sQ04o0eeQEySLrcz4DsX3saHfsr8/N6tfhblxgZKXxMT2Louargg12oGNfoTRLV09GXhVUe5/qgA5vdgNigg==", + "dependencies": { + "adjust-sourcemap-loader": "3.0.0", + "camelcase": "5.3.1", + "compose-function": "3.0.3", + "convert-source-map": "1.7.0", + "es6-iterator": "2.0.3", + "loader-utils": "1.2.3", + "postcss": "7.0.36", + "rework": "1.0.1", + "rework-visit": "1.0.0", + "source-map": "0.6.1" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/resolve-url-loader/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmmirror.com/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/resolve-url-loader/node_modules/convert-source-map": { + "version": "1.7.0", + "resolved": "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-1.7.0.tgz", + "integrity": "sha512-4FJkXzKXEDB1snCFZlLP4gpC3JILicCpGbzG9f9G7tGqGCzETQ2hWPrcinA9oU4wtf2biUaEH5065UnMeR33oA==", + "dependencies": { + "safe-buffer": "~5.1.1" + } + }, + "node_modules/resolve-url-loader/node_modules/emojis-list": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/emojis-list/-/emojis-list-2.1.0.tgz", + "integrity": "sha512-knHEZMgs8BB+MInokmNTg/OyPlAddghe1YBgNwJBc5zsJi/uyIcXoSDsL/W9ymOsBoBGdPIHXYJ9+qKFwRwDng==", + "engines": { + "node": ">= 0.10" + } + }, + "node_modules/resolve-url-loader/node_modules/json5": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "dependencies": { + "minimist": "^1.2.0" + }, + "bin": { + "json5": "lib/cli.js" + } + }, + "node_modules/resolve-url-loader/node_modules/loader-utils": { + "version": "1.2.3", + "resolved": "https://registry.npmmirror.com/loader-utils/-/loader-utils-1.2.3.tgz", + "integrity": "sha512-fkpz8ejdnEMG3s37wGL07iSBDg99O9D5yflE9RGNH3hRdx9SOwYfnGYdZOUIZitN8E+E2vkq3MUMYMvPYl5ZZA==", + "dependencies": { + "big.js": "^5.2.2", + "emojis-list": "^2.0.0", + "json5": "^1.0.1" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/resolve-url-loader/node_modules/postcss": { + "version": "7.0.36", + "resolved": "https://registry.npmmirror.com/postcss/-/postcss-7.0.36.tgz", + "integrity": "sha512-BebJSIUMwJHRH0HAQoxN4u1CN86glsrwsW0q7T+/m44eXOUAxSNdHRkNZPYz5vVUbg17hFgOQDE7fZk7li3pZw==", + "dependencies": { + "chalk": "^2.4.2", + "source-map": "^0.6.1", + "supports-color": "^6.1.0" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/resolve-url-loader/node_modules/postcss/node_modules/supports-color": { + "version": "6.1.0", + "resolved": "https://registry.npmmirror.com/supports-color/-/supports-color-6.1.0.tgz", + "integrity": "sha512-qe1jfm1Mg7Nq/NSh6XE24gPXROEVsWHxC1LIx//XNlD9iw7YZQGjZNjYN7xGaEG6iKdA8EtNFW6R0gjnVXp+wQ==", + "dependencies": { + "has-flag": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/resolve-url-loader/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/responselike": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-2.0.0.tgz", + "integrity": "sha512-xH48u3FTB9VsZw7R+vvgaKeLKzT6jOogbQhEe/jewwnZgzPcnyWui2Av6JpoYZF/91uueC+lqhWqeURw5/qhCw==", + "dependencies": { + "lowercase-keys": "^2.0.0" + } + }, + "node_modules/restore-cursor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/restore-cursor/-/restore-cursor-3.1.0.tgz", + "integrity": "sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA==", + "dependencies": { + "onetime": "^5.1.0", + "signal-exit": "^3.0.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/retext-english": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/retext-english/-/retext-english-3.0.4.tgz", + "integrity": "sha512-yr1PgaBDde+25aJXrnt3p1jvT8FVLVat2Bx8XeAWX13KXo8OT+3nWGU3HWxM4YFJvmfqvJYJZG2d7xxaO774gw==", + "dependencies": { + "parse-english": "^4.0.0", + "unherit": "^1.0.4" + } + }, + "node_modules/retry": { + "version": "0.12.0", + "resolved": "https://registry.npmjs.org/retry/-/retry-0.12.0.tgz", + "integrity": "sha1-G0KmJmoh8HQh0bC1S33BZ7AcATs=", + "engines": { + "node": ">= 4" + } + }, + "node_modules/reusify": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/reusify/-/reusify-1.0.4.tgz", + "integrity": "sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw==", + "engines": { + "iojs": ">=1.0.0", + "node": ">=0.10.0" + } + }, + "node_modules/rework": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/rework/-/rework-1.0.1.tgz", + "integrity": "sha512-eEjL8FdkdsxApd0yWVZgBGzfCQiT8yqSc2H1p4jpZpQdtz7ohETiDMoje5PlM8I9WgkqkreVxFUKYOiJdVWDXw==", + "dependencies": { + "convert-source-map": "^0.3.3", + "css": "^2.0.0" + } + }, + "node_modules/rework-visit": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/rework-visit/-/rework-visit-1.0.0.tgz", + "integrity": "sha512-W6V2fix7nCLUYX1v6eGPrBOZlc03/faqzP4sUxMAJMBMOPYhfV/RyLegTufn5gJKaOITyi+gvf0LXDZ9NzkHnQ==" + }, + "node_modules/rework/node_modules/convert-source-map": { + "version": "0.3.5", + "resolved": "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-0.3.5.tgz", + "integrity": "sha512-+4nRk0k3oEpwUB7/CalD7xE2z4VmtEnnq0GO2IPTkrooTrAhEsWvuLF5iWP1dXrwluki/azwXV1ve7gtYuPldg==" + }, + "node_modules/rimraf": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz", + "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==", + "dependencies": { + "glob": "^7.1.3" + }, + "bin": { + "rimraf": "bin.js" + } + }, + "node_modules/run-async": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/run-async/-/run-async-2.4.1.tgz", + "integrity": "sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ==", + "engines": { + "node": ">=0.12.0" + } + }, + "node_modules/run-parallel": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/run-parallel/-/run-parallel-1.2.0.tgz", + "integrity": "sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA==", + "dependencies": { + "queue-microtask": "^1.2.2" + } + }, + "node_modules/rxjs": { + "version": "6.6.7", + "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-6.6.7.tgz", + "integrity": "sha512-hTdwr+7yYNIT5n4AMYp85KA6yw2Va0FLa3Rguvbpa4W3I5xynaBZo41cM3XM+4Q6fRMj3sBYIR1VAmZMXYJvRQ==", + "dependencies": { + "tslib": "^1.9.0" + }, + "engines": { + "npm": ">=2.0.0" + } + }, + "node_modules/rxjs/node_modules/tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + }, + "node_modules/safe-buffer": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.1.2.tgz", + "integrity": "sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g==" + }, + "node_modules/safer-buffer": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", + "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==" + }, + "node_modules/sanitize-html": { + "version": "1.27.5", + "resolved": "https://registry.npmmirror.com/sanitize-html/-/sanitize-html-1.27.5.tgz", + "integrity": "sha512-M4M5iXDAUEcZKLXkmk90zSYWEtk5NH3JmojQxKxV371fnMh+x9t1rqdmXaGoyEHw3z/X/8vnFhKjGL5xFGOJ3A==", + "dependencies": { + "htmlparser2": "^4.1.0", + "lodash": "^4.17.15", + "parse-srcset": "^1.0.2", + "postcss": "^7.0.27" + } + }, + "node_modules/sanitize-html/node_modules/domhandler": { + "version": "3.3.0", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-3.3.0.tgz", + "integrity": "sha512-J1C5rIANUbuYK+FuFL98650rihynUOEzRLxW+90bKZRWB6A1X1Tf82GxR1qAWLyfNPRvjqfip3Q5tdYlmAa9lA==", + "dependencies": { + "domelementtype": "^2.0.1" + }, + "engines": { + "node": ">= 4" + } + }, + "node_modules/sanitize-html/node_modules/htmlparser2": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/htmlparser2/-/htmlparser2-4.1.0.tgz", + "integrity": "sha512-4zDq1a1zhE4gQso/c5LP1OtrhYTncXNSpvJYtWJBtXAETPlMfi3IFNjGuQbYLuVY4ZR0QMqRVvo4Pdy9KLyP8Q==", + "dependencies": { + "domelementtype": "^2.0.1", + "domhandler": "^3.0.0", + "domutils": "^2.0.0", + "entities": "^2.0.0" + } + }, + "node_modules/sanitize-html/node_modules/picocolors": { + "version": "0.2.1", + "resolved": "https://registry.npmmirror.com/picocolors/-/picocolors-0.2.1.tgz", + "integrity": "sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA==" + }, + "node_modules/sanitize-html/node_modules/postcss": { + "version": "7.0.39", + "resolved": "https://registry.npmmirror.com/postcss/-/postcss-7.0.39.tgz", + "integrity": "sha512-yioayjNbHn6z1/Bywyb2Y4s3yvDAeXGOyxqD+LnVOinq6Mdmd++SW2wUNVzavyyHxd6+DxzWGIuosg6P1Rj8uA==", + "dependencies": { + "picocolors": "^0.2.1", + "source-map": "^0.6.1" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/sanitize-html/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/sass": { + "version": "1.51.0", + "resolved": "https://registry.npmmirror.com/sass/-/sass-1.51.0.tgz", + "integrity": "sha512-haGdpTgywJTvHC2b91GSq+clTKGbtkkZmVAb82jZQN/wTy6qs8DdFm2lhEQbEwrY0QDRgSQ3xDurqM977C3noA==", + "peer": true, + "dependencies": { + "chokidar": ">=3.0.0 <4.0.0", + "immutable": "^4.0.0", + "source-map-js": ">=0.6.2 <2.0.0" + }, + "bin": { + "sass": "sass.js" + }, + "engines": { + "node": ">=12.0.0" + } + }, + "node_modules/sass-graph": { + "version": "2.2.5", + "resolved": "https://registry.npmmirror.com/sass-graph/-/sass-graph-2.2.5.tgz", + "integrity": "sha512-VFWDAHOe6mRuT4mZRd4eKE+d8Uedrk6Xnh7Sh9b4NGufQLQjOrvf/MQoOdx+0s92L89FeyUUNfU597j/3uNpag==", + "dependencies": { + "glob": "^7.0.0", + "lodash": "^4.0.0", + "scss-tokenizer": "^0.2.3", + "yargs": "^13.3.2" + }, + "bin": { + "sassgraph": "bin/sassgraph" + } + }, + "node_modules/sass-graph/node_modules/ansi-regex": { + "version": "4.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-4.1.1.tgz", + "integrity": "sha512-ILlv4k/3f6vfQ4OoP2AGvirOktlQ98ZEL1k9FaQjxa3L1abBgbuTDAdPOpvbGncC0BTVQrl+OM8xZGK6tWXt7g==", + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/cliui": { + "version": "5.0.0", + "resolved": "https://registry.npmmirror.com/cliui/-/cliui-5.0.0.tgz", + "integrity": "sha512-PYeGSEmmHM6zvoef2w8TPzlrnNpXIjTipYK780YswmIP9vjxmd6Y2a3CB2Ks6/AU8NHjZugXvo8w3oWM2qnwXA==", + "dependencies": { + "string-width": "^3.1.0", + "strip-ansi": "^5.2.0", + "wrap-ansi": "^5.1.0" + } + }, + "node_modules/sass-graph/node_modules/emoji-regex": { + "version": "7.0.3", + "resolved": "https://registry.npmmirror.com/emoji-regex/-/emoji-regex-7.0.3.tgz", + "integrity": "sha512-CwBLREIQ7LvYFB0WyRvwhq5N5qPhc6PMjD6bYggFlI5YyDgl+0vxq5VHbMOFqLg7hfWzmu8T5Z1QofhmTIhItA==" + }, + "node_modules/sass-graph/node_modules/find-up": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/find-up/-/find-up-3.0.0.tgz", + "integrity": "sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg==", + "dependencies": { + "locate-path": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/is-fullwidth-code-point": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz", + "integrity": "sha512-VHskAKYM8RfSFXwee5t5cbN5PZeq1Wrh6qd5bkyiXIf6UQcN6w/A0eXM9r6t8d+GYOh+o6ZhiEnb88LN/Y8m2w==", + "engines": { + "node": ">=4" + } + }, + "node_modules/sass-graph/node_modules/locate-path": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/locate-path/-/locate-path-3.0.0.tgz", + "integrity": "sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A==", + "dependencies": { + "p-locate": "^3.0.0", + "path-exists": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/p-locate": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/p-locate/-/p-locate-3.0.0.tgz", + "integrity": "sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ==", + "dependencies": { + "p-limit": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha512-bpC7GYwiDYQ4wYLe+FA8lhRjhQCMcQGuSgGGqDkg/QerRWw9CmGRT0iSOVRSZJ29NMLZgIzqaljJ63oaL4NIJQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/sass-graph/node_modules/string-width": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/string-width/-/string-width-3.1.0.tgz", + "integrity": "sha512-vafcv6KjVZKSgz06oM/H6GDBrAtz8vdhQakGjFIvNrHA6y3HCF1CInLy+QLq8dTJPQ1b+KDUqDFctkdRW44e1w==", + "dependencies": { + "emoji-regex": "^7.0.1", + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^5.1.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/strip-ansi": { + "version": "5.2.0", + "resolved": "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-5.2.0.tgz", + "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", + "dependencies": { + "ansi-regex": "^4.1.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/wrap-ansi": { + "version": "5.1.0", + "resolved": "https://registry.npmmirror.com/wrap-ansi/-/wrap-ansi-5.1.0.tgz", + "integrity": "sha512-QC1/iN/2/RPVJ5jYK8BGttj5z83LmSKmvbvrXPNCLZSEb32KKVDJDl/MOt2N01qU2H/FkzEa9PKto1BqDjtd7Q==", + "dependencies": { + "ansi-styles": "^3.2.0", + "string-width": "^3.0.0", + "strip-ansi": "^5.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/sass-graph/node_modules/yargs": { + "version": "13.3.2", + "resolved": "https://registry.npmmirror.com/yargs/-/yargs-13.3.2.tgz", + "integrity": "sha512-AX3Zw5iPruN5ie6xGRIDgqkT+ZhnRlZMLMHAs8tg7nRruy2Nb+i5o9bwghAogtM08q1dpr2LVoS8KSTMYpWXUw==", + "dependencies": { + "cliui": "^5.0.0", + "find-up": "^3.0.0", + "get-caller-file": "^2.0.1", + "require-directory": "^2.1.1", + "require-main-filename": "^2.0.0", + "set-blocking": "^2.0.0", + "string-width": "^3.0.0", + "which-module": "^2.0.0", + "y18n": "^4.0.0", + "yargs-parser": "^13.1.2" + } + }, + "node_modules/sass-graph/node_modules/yargs-parser": { + "version": "13.1.2", + "resolved": "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-13.1.2.tgz", + "integrity": "sha512-3lbsNRf/j+A4QuSZfDRA7HRSfWrzO0YjqTJd5kjAq37Zep1CEgaYmrH9Q3GwPiB9cHyd1Y1UwggGhJGoxipbzg==", + "dependencies": { + "camelcase": "^5.0.0", + "decamelize": "^1.2.0" + } + }, + "node_modules/sass-loader": { + "version": "10.2.1", + "resolved": "https://registry.npmmirror.com/sass-loader/-/sass-loader-10.2.1.tgz", + "integrity": "sha512-RRvWl+3K2LSMezIsd008ErK4rk6CulIMSwrcc2aZvjymUgKo/vjXGp1rSWmfTUX7bblEOz8tst4wBwWtCGBqKA==", + "dependencies": { + "klona": "^2.0.4", + "loader-utils": "^2.0.0", + "neo-async": "^2.6.2", + "schema-utils": "^3.0.0", + "semver": "^7.3.2" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "fibers": ">= 3.1.0", + "node-sass": "^4.0.0 || ^5.0.0 || ^6.0.0", + "sass": "^1.3.0", + "webpack": "^4.36.0 || ^5.0.0" + }, + "peerDependenciesMeta": { + "fibers": { + "optional": true + }, + "node-sass": { + "optional": true + }, + "sass": { + "optional": true + } + } + }, + "node_modules/sass-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/sass/node_modules/immutable": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/immutable/-/immutable-4.0.0.tgz", + "integrity": "sha512-zIE9hX70qew5qTUjSS7wi1iwj/l7+m54KWU247nhM3v806UdGj1yDndXj+IOYxxtW9zyLI+xqFNZjTuDaLUqFw==", + "peer": true + }, + "node_modules/sax": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/sax/-/sax-1.2.4.tgz", + "integrity": "sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw==" + }, + "node_modules/scheduler": { + "version": "0.20.1", + "integrity": "sha512-LKTe+2xNJBNxu/QhHvDR14wUXHRQbVY5ZOYpOGWRzhydZUqrLb2JBvLPY7cAqFmqrWuDED0Mjk7013SZiOz6Bw==", + "dependencies": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1" + } + }, + "node_modules/schema-utils": { + "version": "2.7.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-2.7.1.tgz", + "integrity": "sha512-SHiNtMOUGWBQJwzISiVYKu82GiV4QYGePp3odlY1tuKO7gPtphAT5R/py0fA6xtbgLL/RvtJZnU9b8s0F1q0Xg==", + "dependencies": { + "@types/json-schema": "^7.0.5", + "ajv": "^6.12.4", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 8.9.0" + } + }, + "node_modules/scss-tokenizer": { + "version": "0.2.3", + "resolved": "https://registry.npmmirror.com/scss-tokenizer/-/scss-tokenizer-0.2.3.tgz", + "integrity": "sha512-dYE8LhncfBUar6POCxMTm0Ln+erjeczqEvCJib5/7XNkdw1FkUGgwMPY360FY0FgPWQxHWCx29Jl3oejyGLM9Q==", + "dependencies": { + "js-base64": "^2.1.8", + "source-map": "^0.4.2" + } + }, + "node_modules/scss-tokenizer/node_modules/source-map": { + "version": "0.4.4", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.4.4.tgz", + "integrity": "sha512-Y8nIfcb1s/7DcobUz1yOO1GSp7gyL+D9zLHDehT7iRESqGSxjJ448Sg7rvfgsRJCnKLdSl11uGf0s9X80cH0/A==", + "dependencies": { + "amdefine": ">=0.0.4" + }, + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/section-matter": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/section-matter/-/section-matter-1.0.0.tgz", + "integrity": "sha512-vfD3pmTzGpufjScBh50YHKzEu2lxBWhVEHsNGoEXmCmn2hKGfeNLYMzCJpe8cD7gqX7TJluOVpBkAequ6dgMmA==", + "dependencies": { + "extend-shallow": "^2.0.1", + "kind-of": "^6.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/semver": { + "version": "7.3.7", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.7.tgz", + "integrity": "sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g==", + "dependencies": { + "lru-cache": "^6.0.0" + }, + "bin": { + "semver": "bin/semver.js" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/semver-diff": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/semver-diff/-/semver-diff-3.1.1.tgz", + "integrity": "sha512-GX0Ix/CJcHyB8c4ykpHGIAvLyOwOobtM/8d+TQkAd81/bEjgPHrfba41Vpesr7jX/t8Uh+R3EX9eAS5be+jQYg==", + "dependencies": { + "semver": "^6.3.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/semver-diff/node_modules/semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/send": { + "version": "0.18.0", + "resolved": "https://registry.npmjs.org/send/-/send-0.18.0.tgz", + "integrity": "sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg==", + "dependencies": { + "debug": "2.6.9", + "depd": "2.0.0", + "destroy": "1.2.0", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "etag": "~1.8.1", + "fresh": "0.5.2", + "http-errors": "2.0.0", + "mime": "1.6.0", + "ms": "2.1.3", + "on-finished": "2.4.1", + "range-parser": "~1.2.1", + "statuses": "2.0.1" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/send/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/send/node_modules/debug/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/send/node_modules/mime": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-1.6.0.tgz", + "integrity": "sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg==", + "bin": { + "mime": "cli.js" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/send/node_modules/ms": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==" + }, + "node_modules/sentence-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/sentence-case/-/sentence-case-3.0.4.tgz", + "integrity": "sha512-8LS0JInaQMCRoQ7YUytAo/xUu5W2XnQxV2HI/6uM6U7CITS1RqPElr30V6uIqyMKM9lJGRVFy5/4CuzcixNYSg==", + "dependencies": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case-first": "^2.0.2" + } + }, + "node_modules/serialize-javascript": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-5.0.1.tgz", + "integrity": "sha512-SaaNal9imEO737H2c05Og0/8LUXG7EnsZyMa8MzkmuHoELfT6txuj0cMqRj6zfPKnmQ1yasR4PCJc8x+M4JSPA==", + "dependencies": { + "randombytes": "^2.1.0" + } + }, + "node_modules/serve-static": { + "version": "1.15.0", + "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.15.0.tgz", + "integrity": "sha512-XGuRDNjXUijsUL0vl6nSD7cwURuzEgglbOaFuZM9g3kwDXOWVTck0jLzjPzGD+TazWbboZYu52/9/XPdUgne9g==", + "dependencies": { + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "parseurl": "~1.3.3", + "send": "0.18.0" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/set-blocking": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/set-blocking/-/set-blocking-2.0.0.tgz", + "integrity": "sha1-BF+XgtARrppoA93TgrJDkrPYkPc=" + }, + "node_modules/setimmediate": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/setimmediate/-/setimmediate-1.0.5.tgz", + "integrity": "sha1-KQy7Iy4waULX1+qbg3Mqt4VvgoU=" + }, + "node_modules/setprototypeof": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/setprototypeof/-/setprototypeof-1.2.0.tgz", + "integrity": "sha512-E5LDX7Wrp85Kil5bhZv46j8jOeboKq5JMmYM3gVGdGH8xFpPWXUMsNrlODCrkoxMEeNi/XZIwuRvY4XNwYMJpw==" + }, + "node_modules/shallow-clone": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/shallow-clone/-/shallow-clone-3.0.1.tgz", + "integrity": "sha512-/6KqX+GVUdqPuPPd2LxDDxzX6CAbjJehAAOKlNpqqUpAqPM6HeL8f+o3a+JsyGjn2lv0WY8UsTgUJjU9Ok55NA==", + "dependencies": { + "kind-of": "^6.0.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/shallow-compare": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/shallow-compare/-/shallow-compare-1.2.2.tgz", + "integrity": "sha512-LUMFi+RppPlrHzbqmFnINTrazo0lPNwhcgzuAXVVcfy/mqPDrQmHAyz5bvV0gDAuRFrk804V0HpQ6u9sZ0tBeg==" + }, + "node_modules/sharp": { + "version": "0.30.4", + "resolved": "https://registry.npmjs.org/sharp/-/sharp-0.30.4.tgz", + "integrity": "sha512-3Onig53Y6lji4NIZo69s14mERXXY/GV++6CzOYx/Rd8bnTwbhFbL09WZd7Ag/CCnA0WxFID8tkY0QReyfL6v0Q==", + "hasInstallScript": true, + "dependencies": { + "color": "^4.2.3", + "detect-libc": "^2.0.1", + "node-addon-api": "^4.3.0", + "prebuild-install": "^7.0.1", + "semver": "^7.3.7", + "simple-get": "^4.0.1", + "tar-fs": "^2.1.1", + "tunnel-agent": "^0.6.0" + }, + "engines": { + "node": ">=12.13.0" + } + }, + "node_modules/sharp/node_modules/detect-libc": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.1.tgz", + "integrity": "sha512-463v3ZeIrcWtdgIg6vI6XUncguvr2TnGl4SzDXinkt9mSLpBJKXT3mW6xT3VQdDN11+WVs29pgvivTc4Lp8v+w==", + "engines": { + "node": ">=8" + } + }, + "node_modules/sharp/node_modules/node-addon-api": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/node-addon-api/-/node-addon-api-4.3.0.tgz", + "integrity": "sha512-73sE9+3UaLYYFmDsFZnqCInzPyh3MqIwZO9cw58yIqAZhONrrabrYyYe3TuIqtIiOuTXVhsGau8hcrhhwSsDIQ==" + }, + "node_modules/shebang-command": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-1.2.0.tgz", + "integrity": "sha1-RKrGW2lbAzmJaMOfNj/uXer98eo=", + "dependencies": { + "shebang-regex": "^1.0.0" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/shebang-regex": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-1.0.0.tgz", + "integrity": "sha1-2kL0l0DAtC2yypcoVxyxkMmO/qM=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/shell-quote": { + "version": "1.7.3", + "resolved": "https://registry.npmjs.org/shell-quote/-/shell-quote-1.7.3.tgz", + "integrity": "sha512-Vpfqwm4EnqGdlsBFNmHhxhElJYrdfcxPThu+ryKS5J8L/fhAwLazFZtq+S+TWZ9ANj2piSQLGj6NQg+lKPmxrw==" + }, + "node_modules/side-channel": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/side-channel/-/side-channel-1.0.4.tgz", + "integrity": "sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw==", + "dependencies": { + "call-bind": "^1.0.0", + "get-intrinsic": "^1.0.2", + "object-inspect": "^1.9.0" + } + }, + "node_modules/signal-exit": { + "version": "3.0.7", + "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.7.tgz", + "integrity": "sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==" + }, + "node_modules/signedsource": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/signedsource/-/signedsource-1.0.0.tgz", + "integrity": "sha1-HdrOSYF5j5O9gzlzgD2A1S6TrWo=" + }, + "node_modules/simple-concat": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/simple-concat/-/simple-concat-1.0.1.tgz", + "integrity": "sha512-cSFtAPtRhljv69IK0hTVZQ+OfE9nePi/rtJmw5UjHeVyVroEqJXP1sFztKUy1qU+xvz3u/sfYJLa947b7nAN2Q==" + }, + "node_modules/simple-get": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/simple-get/-/simple-get-4.0.1.tgz", + "integrity": "sha512-brv7p5WgH0jmQJr1ZDDfKDOSeWWg+OVypG99A/5vYGPqJ6pxiaHLy8nxtFjBA7oMa01ebA9gfh1uMCFqOuXxvA==", + "dependencies": { + "decompress-response": "^6.0.0", + "once": "^1.3.1", + "simple-concat": "^1.0.0" + } + }, + "node_modules/simple-swizzle": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/simple-swizzle/-/simple-swizzle-0.2.2.tgz", + "integrity": "sha1-pNprY1/8zMoz9w0Xy5JZLeleVXo=", + "dependencies": { + "is-arrayish": "^0.3.1" + } + }, + "node_modules/simple-swizzle/node_modules/is-arrayish": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.3.2.tgz", + "integrity": "sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ==" + }, + "node_modules/sisteransi": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", + "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==" + }, + "node_modules/slash": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", + "integrity": "sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/slice-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/slice-ansi/-/slice-ansi-4.0.0.tgz", + "integrity": "sha512-qMCMfhY040cVHT43K9BFygqYbUPFZKHOg7K73mtTWJRb8pyP3fzf4Ixd5SzdEJQ6MRUg/WBnOLxghZtKKurENQ==", + "dependencies": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/slice-ansi/node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/slice-ansi/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/slice-ansi/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/slugify": { + "version": "1.6.5", + "resolved": "https://registry.npmjs.org/slugify/-/slugify-1.6.5.tgz", + "integrity": "sha512-8mo9bslnBO3tr5PEVFzMPIWwWnipGS0xVbYf65zxDqfNwmzYn1LpiKNrR6DlClusuvo+hDHd1zKpmfAe83NQSQ==", + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/snake-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/snake-case/-/snake-case-3.0.4.tgz", + "integrity": "sha512-LAOh4z89bGQvl9pFfNF8V146i7o7/CqFPbqzYgP+yYzDIDeS9HaNFtXABamRW+AQzEVODcvE79ljJ+8a9YSdMg==", + "dependencies": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "node_modules/socket.io": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/socket.io/-/socket.io-3.1.2.tgz", + "integrity": "sha512-JubKZnTQ4Z8G4IZWtaAZSiRP3I/inpy8c/Bsx2jrwGrTbKeVU5xd6qkKMHpChYeM3dWZSO0QACiGK+obhBNwYw==", + "dependencies": { + "@types/cookie": "^0.4.0", + "@types/cors": "^2.8.8", + "@types/node": ">=10.0.0", + "accepts": "~1.3.4", + "base64id": "~2.0.0", + "debug": "~4.3.1", + "engine.io": "~4.1.0", + "socket.io-adapter": "~2.1.0", + "socket.io-parser": "~4.0.3" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/socket.io-adapter": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/socket.io-adapter/-/socket.io-adapter-2.1.0.tgz", + "integrity": "sha512-+vDov/aTsLjViYTwS9fPy5pEtTkrbEKsw2M+oVSoFGw6OD1IpvlV1VPhUzNbofCQ8oyMbdYJqDtGdmHQK6TdPg==" + }, + "node_modules/socket.io-client": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/socket.io-client/-/socket.io-client-3.1.3.tgz", + "integrity": "sha512-4sIGOGOmCg3AOgGi7EEr6ZkTZRkrXwub70bBB/F0JSkMOUFpA77WsL87o34DffQQ31PkbMUIadGOk+3tx1KGbw==", + "dependencies": { + "@types/component-emitter": "^1.2.10", + "backo2": "~1.0.2", + "component-emitter": "~1.3.0", + "debug": "~4.3.1", + "engine.io-client": "~4.1.0", + "parseuri": "0.0.6", + "socket.io-parser": "~4.0.4" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/socket.io-client/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/socket.io-parser": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/socket.io-parser/-/socket.io-parser-4.0.4.tgz", + "integrity": "sha512-t+b0SS+IxG7Rxzda2EVvyBZbvFPBCjJoyHuE0P//7OAsN23GItzDRdWa6ALxZI/8R5ygK7jAR6t028/z+7295g==", + "dependencies": { + "@types/component-emitter": "^1.2.10", + "component-emitter": "~1.3.0", + "debug": "~4.3.1" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/socket.io-parser/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/socket.io/node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/source-list-map": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/source-list-map/-/source-list-map-2.0.1.tgz", + "integrity": "sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw==" + }, + "node_modules/source-map": { + "version": "0.7.3", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.7.3.tgz", + "integrity": "sha512-CkCj6giN3S+n9qrYiBTX5gystlENnRW5jZeNLHpe6aue+SrHcG5VYwujhW9s4dY31mEGsxBDrHR6oI69fTXsaQ==", + "engines": { + "node": ">= 8" + } + }, + "node_modules/source-map-js": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/source-map-js/-/source-map-js-1.0.2.tgz", + "integrity": "sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/source-map-resolve": { + "version": "0.5.3", + "resolved": "https://registry.npmmirror.com/source-map-resolve/-/source-map-resolve-0.5.3.tgz", + "integrity": "sha512-Htz+RnsXWk5+P2slx5Jh3Q66vhQj1Cllm0zvnaY98+NFx+Dv2CF/f5O/t8x+KaNdrdIAsruNzoh/KpialbqAnw==", + "deprecated": "See https://github.com/lydell/source-map-resolve#deprecated", + "dependencies": { + "atob": "^2.1.2", + "decode-uri-component": "^0.2.0", + "resolve-url": "^0.2.1", + "source-map-url": "^0.4.0", + "urix": "^0.1.0" + } + }, + "node_modules/source-map-support": { + "version": "0.5.21", + "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.5.21.tgz", + "integrity": "sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w==", + "dependencies": { + "buffer-from": "^1.0.0", + "source-map": "^0.6.0" + } + }, + "node_modules/source-map-support/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/source-map-url": { + "version": "0.4.1", + "resolved": "https://registry.npmmirror.com/source-map-url/-/source-map-url-0.4.1.tgz", + "integrity": "sha512-cPiFOTLUKvJFIg4SKVScy4ilPPW6rFgMgfuZJPNoDuMs3nC1HbMUycBoJw77xFIp6z1UJQJOfx6C9GMH80DiTw==", + "deprecated": "See https://github.com/lydell/source-map-url#deprecated" + }, + "node_modules/space-separated-tokens": { + "version": "1.1.5", + "resolved": "https://registry.npmmirror.com/space-separated-tokens/-/space-separated-tokens-1.1.5.tgz", + "integrity": "sha512-q/JSVd1Lptzhf5bkYm4ob4iWPjx0KiRe3sRFBNrVqbJkFaBm5vbbowy1mymoPNLRa52+oadOhJ+K49wsSeSjTA==" + }, + "node_modules/spdx-correct": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/spdx-correct/-/spdx-correct-3.1.1.tgz", + "integrity": "sha512-cOYcUWwhCuHCXi49RhFRCyJEK3iPj1Ziz9DpViV3tbZOwXD49QzIN3MpOLJNxh2qwq2lJJZaKMVw9qNi4jTC0w==", + "dependencies": { + "spdx-expression-parse": "^3.0.0", + "spdx-license-ids": "^3.0.0" + } + }, + "node_modules/spdx-exceptions": { + "version": "2.3.0", + "resolved": "https://registry.npmmirror.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz", + "integrity": "sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A==" + }, + "node_modules/spdx-expression-parse": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz", + "integrity": "sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q==", + "dependencies": { + "spdx-exceptions": "^2.1.0", + "spdx-license-ids": "^3.0.0" + } + }, + "node_modules/spdx-license-ids": { + "version": "3.0.11", + "resolved": "https://registry.npmmirror.com/spdx-license-ids/-/spdx-license-ids-3.0.11.tgz", + "integrity": "sha512-Ctl2BrFiM0X3MANYgj3CkygxhRmr9mi6xhejbdO960nF6EDJApTYpn0BQnDKlnNBULKiCN1n3w9EBkHK8ZWg+g==" + }, + "node_modules/split-on-first": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/split-on-first/-/split-on-first-1.1.0.tgz", + "integrity": "sha512-43ZssAJaMusuKWL8sKUBQXHWOpq8d6CfN/u1p4gUzfJkM05C8rxTmYrkIPTXapZpORA6LkkzcUulJ8FqA7Uudw==", + "engines": { + "node": ">=6" + } + }, + "node_modules/sponge-case": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/sponge-case/-/sponge-case-1.0.1.tgz", + "integrity": "sha512-dblb9Et4DAtiZ5YSUZHLl4XhH4uK80GhAZrVXdN4O2P4gQ40Wa5UIOPUHlA/nFd2PLblBZWUioLMMAVrgpoYcA==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/sprintf-js": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", + "integrity": "sha1-BOaSb2YolTVPPdAVIDYzuFcpfiw=" + }, + "node_modules/sshpk": { + "version": "1.17.0", + "resolved": "https://registry.npmmirror.com/sshpk/-/sshpk-1.17.0.tgz", + "integrity": "sha512-/9HIEs1ZXGhSPE8X6Ccm7Nam1z8KcoCqPdI7ecm1N33EzAetWahvQWVqLZtaZQ+IDKX4IyA2o0gBzqIMkAagHQ==", + "dependencies": { + "asn1": "~0.2.3", + "assert-plus": "^1.0.0", + "bcrypt-pbkdf": "^1.0.0", + "dashdash": "^1.12.0", + "ecc-jsbn": "~0.1.1", + "getpass": "^0.1.1", + "jsbn": "~0.1.0", + "safer-buffer": "^2.0.2", + "tweetnacl": "~0.14.0" + }, + "bin": { + "sshpk-conv": "bin/sshpk-conv", + "sshpk-sign": "bin/sshpk-sign", + "sshpk-verify": "bin/sshpk-verify" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/st": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/st/-/st-2.0.0.tgz", + "integrity": "sha512-drN+aGYnrZPNYIymmNwIY7LXYJ8MqsqXj4fMRue3FOgGMdGjSX10fhJ3qx0sVQPhcWxhEaN4U/eWM4O4dbYNAw==", + "dependencies": { + "async-cache": "^1.1.0", + "bl": "^4.0.0", + "fd": "~0.0.2", + "mime": "^2.4.4", + "negotiator": "~0.6.2" + }, + "bin": { + "st": "bin/server.js" + }, + "optionalDependencies": { + "graceful-fs": "^4.2.3" + } + }, + "node_modules/stable": { + "version": "0.1.8", + "resolved": "https://registry.npmjs.org/stable/-/stable-0.1.8.tgz", + "integrity": "sha512-ji9qxRnOVfcuLDySj9qzhGSEFVobyt1kIOSkj1qZzYLzq7Tos/oUUWvotUPQLlrsidqsK6tBH89Bc9kL5zHA6w==" + }, + "node_modules/stack-trace": { + "version": "0.0.10", + "resolved": "https://registry.npmjs.org/stack-trace/-/stack-trace-0.0.10.tgz", + "integrity": "sha1-VHxws0fo0ytOEI6hoqFZ5f3eGcA=", + "engines": { + "node": "*" + } + }, + "node_modules/stackframe": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/stackframe/-/stackframe-1.2.1.tgz", + "integrity": "sha512-h88QkzREN/hy8eRdyNhhsO7RSJ5oyTqxxmmn0dzBIMUclZsjpfmrsg81vp8mjjAs2vAZ72nyWxRUwSwmh0e4xg==" + }, + "node_modules/statuses": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/statuses/-/statuses-2.0.1.tgz", + "integrity": "sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ==", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/stdout-stream": { + "version": "1.4.1", + "resolved": "https://registry.npmmirror.com/stdout-stream/-/stdout-stream-1.4.1.tgz", + "integrity": "sha512-j4emi03KXqJWcIeF8eIXkjMFN1Cmb8gUlDYGeBALLPo5qdyTfA9bOtl8m33lRoC+vFMkP3gl0WsDr6+gzxbbTA==", + "dependencies": { + "readable-stream": "^2.0.1" + } + }, + "node_modules/stdout-stream/node_modules/readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "node_modules/stdout-stream/node_modules/string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "dependencies": { + "safe-buffer": "~5.1.0" + } + }, + "node_modules/stream-parser": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/stream-parser/-/stream-parser-0.3.1.tgz", + "integrity": "sha1-FhhUhpRCACGhGC/wrxkRwSl2F3M=", + "dependencies": { + "debug": "2" + } + }, + "node_modules/stream-parser/node_modules/debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dependencies": { + "ms": "2.0.0" + } + }, + "node_modules/stream-parser/node_modules/ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "node_modules/streamsearch": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/streamsearch/-/streamsearch-0.1.2.tgz", + "integrity": "sha1-gIudDlb8Jz2Am6VzOOkpkZoanxo=", + "engines": { + "node": ">=0.8.0" + } + }, + "node_modules/strict-uri-encode": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/strict-uri-encode/-/strict-uri-encode-2.0.0.tgz", + "integrity": "sha1-ucczDHBChi9rFC3CdLvMWGbONUY=", + "engines": { + "node": ">=4" + } + }, + "node_modules/string_decoder": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz", + "integrity": "sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==", + "dependencies": { + "safe-buffer": "~5.2.0" + } + }, + "node_modules/string_decoder/node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + }, + "node_modules/string-env-interpolation": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/string-env-interpolation/-/string-env-interpolation-1.0.1.tgz", + "integrity": "sha512-78lwMoCcn0nNu8LszbP1UA7g55OeE4v7rCeWnM5B453rnNr4aq+5it3FEYtZrSEiMvHZOZ9Jlqb0OD0M2VInqg==" + }, + "node_modules/string-natural-compare": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/string-natural-compare/-/string-natural-compare-3.0.1.tgz", + "integrity": "sha512-n3sPwynL1nwKi3WJ6AIsClwBMa0zTi54fn2oLU6ndfTSIO05xaznjSf15PcBZU6FNWbmN5Q6cxT4V5hGvB4taw==" + }, + "node_modules/string-similarity": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/string-similarity/-/string-similarity-1.2.2.tgz", + "integrity": "sha512-IoHUjcw3Srl8nsPlW04U3qwWPk3oG2ffLM0tN853d/E/JlIvcmZmDY2Kz5HzKp4lEi2T7QD7Zuvjq/1rDw+XcQ==", + "dependencies": { + "lodash.every": "^4.6.0", + "lodash.flattendeep": "^4.4.0", + "lodash.foreach": "^4.5.0", + "lodash.map": "^4.6.0", + "lodash.maxby": "^4.6.0" + } + }, + "node_modules/string-width": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "dependencies": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/string.prototype.matchall": { + "version": "4.0.7", + "resolved": "https://registry.npmjs.org/string.prototype.matchall/-/string.prototype.matchall-4.0.7.tgz", + "integrity": "sha512-f48okCX7JiwVi1NXCVWcFnZgADDC/n2vePlQ/KUCNqCikLLilQvwjMO8+BHVKvgzH0JB0J9LEPgxOGT02RoETg==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1", + "get-intrinsic": "^1.1.1", + "has-symbols": "^1.0.3", + "internal-slot": "^1.0.3", + "regexp.prototype.flags": "^1.4.1", + "side-channel": "^1.0.4" + } + }, + "node_modules/string.prototype.trimend": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/string.prototype.trimend/-/string.prototype.trimend-1.0.5.tgz", + "integrity": "sha512-I7RGvmjV4pJ7O3kdf+LXFpVfdNOxtCW/2C8f6jNiW4+PQchwxkCDzlk1/7p+Wl4bqFIZeF47qAHXLuHHWKAxog==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "node_modules/string.prototype.trimstart": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/string.prototype.trimstart/-/string.prototype.trimstart-1.0.5.tgz", + "integrity": "sha512-THx16TJCGlsN0o6dl2o6ncWUsdgnLRSA23rRE5pyGBw/mLr3Ej/R2LaqCtgP8VNMGZsvMWnf9ooZPyY2bHvUFg==", + "dependencies": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "node_modules/stringify-entities": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/stringify-entities/-/stringify-entities-3.1.0.tgz", + "integrity": "sha512-3FP+jGMmMV/ffZs86MoghGqAoqXAdxLrJP4GUdrDN1aIScYih5tuIO3eF4To5AJZ79KDZ8Fpdy7QJnK8SsL1Vg==", + "dependencies": { + "character-entities-html4": "^1.0.0", + "character-entities-legacy": "^1.0.0", + "xtend": "^4.0.0" + } + }, + "node_modules/stringify-object": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/stringify-object/-/stringify-object-3.3.0.tgz", + "integrity": "sha512-rHqiFh1elqCQ9WPLIC8I0Q/g/wj5J1eMkyoiD6eoQApWHP0FtlK7rqnhmabL5VUY9JQCcqwwvlOaSuutekgyrw==", + "dependencies": { + "get-own-enumerable-property-symbols": "^3.0.0", + "is-obj": "^1.0.1", + "is-regexp": "^1.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/stringify-object/node_modules/is-obj": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-obj/-/is-obj-1.0.1.tgz", + "integrity": "sha1-PkcprB9f3gJc19g6iW2rn09n2w8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/strip-ansi": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "dependencies": { + "ansi-regex": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/strip-bom": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/strip-bom/-/strip-bom-3.0.0.tgz", + "integrity": "sha1-IzTBjpx1n3vdVv3vfprj1YjmjtM=", + "engines": { + "node": ">=4" + } + }, + "node_modules/strip-bom-string": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/strip-bom-string/-/strip-bom-string-1.0.0.tgz", + "integrity": "sha512-uCC2VHvQRYu+lMh4My/sFNmF2klFymLX1wHJeXnbEJERpV/ZsVuonzerjfrGpIGF7LBVa1O7i9kjiWvJiFck8g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/strip-comments": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/strip-comments/-/strip-comments-1.0.2.tgz", + "integrity": "sha512-kL97alc47hoyIQSV165tTt9rG5dn4w1dNnBhOQ3bOU1Nc1hel09jnXANaHJ7vzHLd4Ju8kseDGzlev96pghLFw==", + "dependencies": { + "babel-extract-comments": "^1.0.0", + "babel-plugin-transform-object-rest-spread": "^6.26.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/strip-eof": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/strip-eof/-/strip-eof-1.0.0.tgz", + "integrity": "sha1-u0P/VZim6wXYm1n80SnJgzE2Br8=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/strip-final-newline": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/strip-final-newline/-/strip-final-newline-2.0.0.tgz", + "integrity": "sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA==", + "engines": { + "node": ">=6" + } + }, + "node_modules/strip-indent": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/strip-indent/-/strip-indent-3.0.0.tgz", + "integrity": "sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ==", + "dependencies": { + "min-indent": "^1.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/strip-json-comments": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", + "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", + "engines": { + "node": ">=8" + } + }, + "node_modules/strip-outer": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/strip-outer/-/strip-outer-1.0.1.tgz", + "integrity": "sha512-k55yxKHwaXnpYGsOzg4Vl8+tDrWylxDEpknGjhTiZB8dFRU5rTo9CAzeycivxV3s+zlTKwrs6WxMxR95n26kwg==", + "dependencies": { + "escape-string-regexp": "^1.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/strtok3": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/strtok3/-/strtok3-6.3.0.tgz", + "integrity": "sha512-fZtbhtvI9I48xDSywd/somNqgUHl2L2cstmXCCif0itOf96jeW18MBSyrLuNicYQVkvpOxkZtkzujiTJ9LW5Jw==", + "dependencies": { + "@tokenizer/token": "^0.3.0", + "peek-readable": "^4.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/style-loader": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/style-loader/-/style-loader-2.0.0.tgz", + "integrity": "sha512-Z0gYUJmzZ6ZdRUqpg1r8GsaFKypE+3xAzuFeMuoHgjc9KZv3wMyCRjQIWEbhoFSq7+7yoHXySDJyyWQaPajeiQ==", + "dependencies": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/style-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/style-to-js": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/style-to-js/-/style-to-js-1.1.0.tgz", + "integrity": "sha512-1OqefPDxGrlMwcbfpsTVRyzwdhr4W0uxYQzeA2F1CBc8WG04udg2+ybRnvh3XYL4TdHQrCahLtax2jc8xaE6rA==", + "dependencies": { + "style-to-object": "0.3.0" + } + }, + "node_modules/style-to-object": { + "version": "0.3.0", + "resolved": "https://registry.npmmirror.com/style-to-object/-/style-to-object-0.3.0.tgz", + "integrity": "sha512-CzFnRRXhzWIdItT3OmF8SQfWyahHhjq3HwcMNCNLn+N7klOOqPjMeG/4JSu77D7ypZdGvSzvkrbyeTMizz2VrA==", + "dependencies": { + "inline-style-parser": "0.1.1" + } + }, + "node_modules/stylehacks": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/stylehacks/-/stylehacks-5.1.0.tgz", + "integrity": "sha512-SzLmvHQTrIWfSgljkQCw2++C9+Ne91d/6Sp92I8c5uHTcy/PgeHamwITIbBW9wnFTY/3ZfSXR9HIL6Ikqmcu6Q==", + "dependencies": { + "browserslist": "^4.16.6", + "postcss-selector-parser": "^6.0.4" + }, + "engines": { + "node": "^10 || ^12 || >=14.0" + }, + "peerDependencies": { + "postcss": "^8.2.15" + } + }, + "node_modules/stylis": { + "version": "4.0.13", + "resolved": "https://registry.npmmirror.com/stylis/-/stylis-4.0.13.tgz", + "integrity": "sha512-xGPXiFVl4YED9Jh7Euv2V220mriG9u4B2TA6Ybjc1catrstKD2PpIdU3U0RKpkVBC2EhmL/F0sPCr9vrFTNRag==" + }, + "node_modules/subscriptions-transport-ws": { + "version": "0.9.19", + "resolved": "https://registry.npmjs.org/subscriptions-transport-ws/-/subscriptions-transport-ws-0.9.19.tgz", + "integrity": "sha512-dxdemxFFB0ppCLg10FTtRqH/31FNRL1y1BQv8209MK5I4CwALb7iihQg+7p65lFcIl8MHatINWBLOqpgU4Kyyw==", + "deprecated": "The `subscriptions-transport-ws` package is no longer maintained. We recommend you use `graphql-ws` instead. For help migrating Apollo software to `graphql-ws`, see https://www.apollographql.com/docs/apollo-server/data/subscriptions/#switching-from-subscriptions-transport-ws For general help using `graphql-ws`, see https://github.com/enisdenjo/graphql-ws/blob/master/README.md", + "dependencies": { + "backo2": "^1.0.2", + "eventemitter3": "^3.1.0", + "iterall": "^1.2.1", + "symbol-observable": "^1.0.4", + "ws": "^5.2.0 || ^6.0.0 || ^7.0.0" + }, + "peerDependencies": { + "graphql": ">=0.10.0" + } + }, + "node_modules/sudo-prompt": { + "version": "8.2.5", + "resolved": "https://registry.npmjs.org/sudo-prompt/-/sudo-prompt-8.2.5.tgz", + "integrity": "sha512-rlBo3HU/1zAJUrkY6jNxDOC9eVYliG6nS4JA8u8KAshITd07tafMc/Br7xQwCSseXwJ2iCcHCE8SNWX3q8Z+kw==" + }, + "node_modules/supports-color": { + "version": "5.5.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", + "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", + "dependencies": { + "has-flag": "^3.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/supports-preserve-symlinks-flag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz", + "integrity": "sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w==", + "engines": { + "node": ">= 0.4" + } + }, + "node_modules/svgo": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/svgo/-/svgo-2.8.0.tgz", + "integrity": "sha512-+N/Q9kV1+F+UeWYoSiULYo4xYSDQlTgb+ayMobAXPwMnLvop7oxKMo9OzIrX5x3eS4L4f2UHhc9axXwY8DpChg==", + "dependencies": { + "@trysound/sax": "0.2.0", + "commander": "^7.2.0", + "css-select": "^4.1.3", + "css-tree": "^1.1.3", + "csso": "^4.2.0", + "picocolors": "^1.0.0", + "stable": "^0.1.8" + }, + "bin": { + "svgo": "bin/svgo" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/swap-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/swap-case/-/swap-case-2.0.2.tgz", + "integrity": "sha512-kc6S2YS/2yXbtkSMunBtKdah4VFETZ8Oh6ONSmSd9bRxhqTrtARUCBUiWXH3xVPpvR7tz2CSnkuXVE42EcGnMw==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/symbol-observable": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/symbol-observable/-/symbol-observable-1.2.0.tgz", + "integrity": "sha512-e900nM8RRtGhlV36KGEU9k65K3mPb1WV70OdjfxlG2EAuM1noi/E/BaW/uMhL7bPEssK8QV57vN3esixjUvcXQ==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/sync-fetch": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/sync-fetch/-/sync-fetch-0.3.0.tgz", + "integrity": "sha512-dJp4qg+x4JwSEW1HibAuMi0IIrBI3wuQr2GimmqB7OXR50wmwzfdusG+p39R9w3R6aFtZ2mzvxvWKQ3Bd/vx3g==", + "dependencies": { + "buffer": "^5.7.0", + "node-fetch": "^2.6.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/table": { + "version": "6.8.0", + "resolved": "https://registry.npmjs.org/table/-/table-6.8.0.tgz", + "integrity": "sha512-s/fitrbVeEyHKFa7mFdkuQMWlH1Wgw/yEXMt5xACT4ZpzWFluehAxRtUUQKPuWhaLAWhFcVx6w3oC8VKaUfPGA==", + "dependencies": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/table/node_modules/ajv": { + "version": "8.11.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.11.0.tgz", + "integrity": "sha512-wGgprdCvMalC0BztXvitD2hC04YffAvtsUn93JbGXYLAtCUO4xd17mCCZQxUOItiBwZvJScWo8NIvQMQ71rdpg==", + "dependencies": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + } + }, + "node_modules/table/node_modules/json-schema-traverse": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz", + "integrity": "sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug==" + }, + "node_modules/tapable": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-2.2.1.tgz", + "integrity": "sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/tar": { + "version": "6.1.11", + "resolved": "https://registry.npmmirror.com/tar/-/tar-6.1.11.tgz", + "integrity": "sha512-an/KZQzQUkZCkuoAA64hM92X0Urb6VpRhAFllDzz44U2mcD5scmT3zBc4VgVpkugF580+DQn8eAFSyoQt0tznA==", + "dependencies": { + "chownr": "^2.0.0", + "fs-minipass": "^2.0.0", + "minipass": "^3.0.0", + "minizlib": "^2.1.1", + "mkdirp": "^1.0.3", + "yallist": "^4.0.0" + }, + "engines": { + "node": ">= 10" + } + }, + "node_modules/tar-fs": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/tar-fs/-/tar-fs-2.1.1.tgz", + "integrity": "sha512-V0r2Y9scmbDRLCNex/+hYzvp/zyYjvFbHPNgVTKfQvVrb6guiE/fxP+XblDNR011utopbkex2nM4dHNV6GDsng==", + "dependencies": { + "chownr": "^1.1.1", + "mkdirp-classic": "^0.5.2", + "pump": "^3.0.0", + "tar-stream": "^2.1.4" + } + }, + "node_modules/tar-stream": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tar-stream/-/tar-stream-2.2.0.tgz", + "integrity": "sha512-ujeqbceABgwMZxEJnk2HDY2DlnUZ+9oEcb1KzTVfYHio0UE6dG71n60d8D2I4qNvleWrrXpmjpt7vZeF1LnMZQ==", + "dependencies": { + "bl": "^4.0.3", + "end-of-stream": "^1.4.1", + "fs-constants": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^3.1.1" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/tar/node_modules/chownr": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/chownr/-/chownr-2.0.0.tgz", + "integrity": "sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ==", + "engines": { + "node": ">=10" + } + }, + "node_modules/tar/node_modules/mkdirp": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/mkdirp/-/mkdirp-1.0.4.tgz", + "integrity": "sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw==", + "bin": { + "mkdirp": "bin/cmd.js" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/term-size": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/term-size/-/term-size-2.2.1.tgz", + "integrity": "sha512-wK0Ri4fOGjv/XPy8SBHZChl8CM7uMc5VML7SqiQ0zG7+J5Vr+RMQDoHa2CNT6KHUnTGIXH34UDMkPzAUyapBZg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/terser": { + "version": "5.13.1", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.13.1.tgz", + "integrity": "sha512-hn4WKOfwnwbYfe48NgrQjqNOH9jzLqRcIfbYytOXCOv46LBfWr9bDS17MQqOi+BWGD0sJK3Sj5NC/gJjiojaoA==", + "dependencies": { + "acorn": "^8.5.0", + "commander": "^2.20.0", + "source-map": "~0.8.0-beta.0", + "source-map-support": "~0.5.20" + }, + "bin": { + "terser": "bin/terser" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/terser-webpack-plugin": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.1.tgz", + "integrity": "sha512-GvlZdT6wPQKbDNW/GDQzZFg/j4vKU96yl2q6mcUkzKOgW4gwf1Z8cZToUCrz31XHlPWH8MVb1r2tFtdDtTGJ7g==", + "dependencies": { + "jest-worker": "^27.4.5", + "schema-utils": "^3.1.1", + "serialize-javascript": "^6.0.0", + "source-map": "^0.6.1", + "terser": "^5.7.2" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "webpack": "^5.1.0" + }, + "peerDependenciesMeta": { + "@swc/core": { + "optional": true + }, + "esbuild": { + "optional": true + }, + "uglify-js": { + "optional": true + } + } + }, + "node_modules/terser-webpack-plugin/node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/terser-webpack-plugin/node_modules/jest-worker": { + "version": "27.5.1", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-27.5.1.tgz", + "integrity": "sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg==", + "dependencies": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^8.0.0" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/terser-webpack-plugin/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/terser-webpack-plugin/node_modules/serialize-javascript": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", + "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "dependencies": { + "randombytes": "^2.1.0" + } + }, + "node_modules/terser-webpack-plugin/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/terser-webpack-plugin/node_modules/supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/terser/node_modules/acorn": { + "version": "8.7.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.7.1.tgz", + "integrity": "sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==", + "bin": { + "acorn": "bin/acorn" + }, + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/terser/node_modules/commander": { + "version": "2.20.3", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", + "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + }, + "node_modules/terser/node_modules/source-map": { + "version": "0.8.0-beta.0", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.8.0-beta.0.tgz", + "integrity": "sha512-2ymg6oRBpebeZi9UUNsgQ89bhx01TcTkmNTGnNO88imTmbSgy4nfujrgVEFKWpMTEGA11EDkTt7mqObTPdigIA==", + "dependencies": { + "whatwg-url": "^7.0.0" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/terser/node_modules/tr46": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-1.0.1.tgz", + "integrity": "sha1-qLE/1r/SSJUZZ0zN5VujaTtwbQk=", + "dependencies": { + "punycode": "^2.1.0" + } + }, + "node_modules/terser/node_modules/webidl-conversions": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-4.0.2.tgz", + "integrity": "sha512-YQ+BmxuTgd6UXZW3+ICGfyqRyHXVlD5GtQr5+qjiNW7bF0cqrzX500HVXPBOvgXb5YnzDd+h0zqyv61KUD7+Sg==" + }, + "node_modules/terser/node_modules/whatwg-url": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-7.1.0.tgz", + "integrity": "sha512-WUu7Rg1DroM7oQvGWfOiAK21n74Gg+T4elXEQYkOhtyLeWiJFoOGLXPKI/9gzIie9CtwVLm8wtw6YJdKyxSjeg==", + "dependencies": { + "lodash.sortby": "^4.7.0", + "tr46": "^1.0.1", + "webidl-conversions": "^4.0.2" + } + }, + "node_modules/text-table": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/text-table/-/text-table-0.2.0.tgz", + "integrity": "sha1-f17oI66AUgfACvLfSoTsP8+lcLQ=" + }, + "node_modules/through": { + "version": "2.3.8", + "resolved": "https://registry.npmjs.org/through/-/through-2.3.8.tgz", + "integrity": "sha1-DdTJ/6q8NXlgsbckEV1+Doai4fU=" + }, + "node_modules/through2": { + "version": "0.4.2", + "resolved": "https://registry.npmmirror.com/through2/-/through2-0.4.2.tgz", + "integrity": "sha512-45Llu+EwHKtAZYTPPVn3XZHBgakWMN3rokhEv5hu596XP+cNgplMg+Gj+1nmAvj+L0K7+N49zBKx5rah5u0QIQ==", + "dependencies": { + "readable-stream": "~1.0.17", + "xtend": "~2.1.1" + } + }, + "node_modules/through2/node_modules/isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmmirror.com/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==" + }, + "node_modules/through2/node_modules/object-keys": { + "version": "0.4.0", + "resolved": "https://registry.npmmirror.com/object-keys/-/object-keys-0.4.0.tgz", + "integrity": "sha512-ncrLw+X55z7bkl5PnUvHwFK9FcGuFYo9gtjws2XtSzL+aZ8tm830P60WJ0dSmFVaSalWieW5MD7kEdnXda9yJw==" + }, + "node_modules/through2/node_modules/readable-stream": { + "version": "1.0.34", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.0.34.tgz", + "integrity": "sha512-ok1qVCJuRkNmvebYikljxJA/UEsKwLl2nI1OmaqAu4/UE+h0wKCHok4XkL/gvi39OacXvw59RJUOFUkDib2rHg==", + "dependencies": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "node_modules/through2/node_modules/string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==" + }, + "node_modules/through2/node_modules/xtend": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/xtend/-/xtend-2.1.2.tgz", + "integrity": "sha512-vMNKzr2rHP9Dp/e1NQFnLQlwlhp9L/LfvnsVdHxN1f+uggyVI3i08uD14GPvCToPkdsRfyPqIyYGmIk58V98ZQ==", + "dependencies": { + "object-keys": "~0.4.0" + }, + "engines": { + "node": ">=0.4" + } + }, + "node_modules/timers-ext": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/timers-ext/-/timers-ext-0.1.7.tgz", + "integrity": "sha512-b85NUNzTSdodShTIbky6ZF02e8STtVVfD+fu4aXXShEELpozH+bCpJLYMPZbsABN2wDH7fJpqIoXxJpzbf0NqQ==", + "dependencies": { + "es5-ext": "~0.10.46", + "next-tick": "1" + } + }, + "node_modules/timm": { + "version": "1.7.1", + "resolved": "https://registry.npmjs.org/timm/-/timm-1.7.1.tgz", + "integrity": "sha512-IjZc9KIotudix8bMaBW6QvMuq64BrJWFs1+4V0lXwWGQZwH+LnX87doAYhem4caOEusRP9/g6jVDQmZ8XOk1nw==" + }, + "node_modules/tinycolor2": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/tinycolor2/-/tinycolor2-1.4.2.tgz", + "integrity": "sha512-vJhccZPs965sV/L2sU4oRQVAos0pQXwsvTLkWYdqJ+a8Q5kPFzJTuOFwy7UniPli44NKQGAglksjvOcpo95aZA==", + "engines": { + "node": "*" + } + }, + "node_modules/title-case": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/title-case/-/title-case-3.0.3.tgz", + "integrity": "sha512-e1zGYRvbffpcHIrnuqT0Dh+gEJtDaxDSoG4JAIpq4oDFyooziLBIiYQv0GBT4FUAnUop5uZ1hiIAj7oAF6sOCA==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/tmp": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.2.1.tgz", + "integrity": "sha512-76SUhtfqR2Ijn+xllcI5P1oyannHNHByD80W1q447gU3mp9G9PSpGdWmjUOHRDPiHYacIk66W7ubDTuPF3BEtQ==", + "dependencies": { + "rimraf": "^3.0.0" + }, + "engines": { + "node": ">=8.17.0" + } + }, + "node_modules/to-fast-properties": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/to-fast-properties/-/to-fast-properties-2.0.0.tgz", + "integrity": "sha1-3F5pjL0HkmW8c+A3doGk5Og/YW4=", + "engines": { + "node": ">=4" + } + }, + "node_modules/to-readable-stream": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/to-readable-stream/-/to-readable-stream-1.0.0.tgz", + "integrity": "sha512-Iq25XBt6zD5npPhlLVXGFN3/gyR2/qODcKNNyTMd4vbm39HUaOiAM4PMq0eMVC/Tkxz+Zjdsc55g9yyz+Yq00Q==", + "engines": { + "node": ">=6" + } + }, + "node_modules/to-regex-range": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", + "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", + "dependencies": { + "is-number": "^7.0.0" + }, + "engines": { + "node": ">=8.0" + } + }, + "node_modules/toidentifier": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/toidentifier/-/toidentifier-1.0.1.tgz", + "integrity": "sha512-o5sSPKEkg/DIQNmH43V0/uerLrpzVedkUh8tGNvaeXpfpuwjKenlSox/2O/BTlZUtEe+JG7s5YhEz608PlAHRA==", + "engines": { + "node": ">=0.6" + } + }, + "node_modules/token-types": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/token-types/-/token-types-4.2.0.tgz", + "integrity": "sha512-P0rrp4wUpefLncNamWIef62J0v0kQR/GfDVji9WKY7GDCWy5YbVSrKUTam07iWPZQGy0zWNOfstYTykMmPNR7w==", + "dependencies": { + "@tokenizer/token": "^0.3.0", + "ieee754": "^1.2.1" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/tough-cookie": { + "version": "2.5.0", + "resolved": "https://registry.npmmirror.com/tough-cookie/-/tough-cookie-2.5.0.tgz", + "integrity": "sha512-nlLsUzgm1kfLXSXfRZMc1KLAugd4hqJHDTvc2hDIwS3mZAfMEuMbc03SujMF+GEcpaX/qboeycw6iO8JwVv2+g==", + "dependencies": { + "psl": "^1.1.28", + "punycode": "^2.1.1" + }, + "engines": { + "node": ">=0.8" + } + }, + "node_modules/tr46": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", + "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" + }, + "node_modules/trim-newlines": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/trim-newlines/-/trim-newlines-3.0.1.tgz", + "integrity": "sha512-c1PTsA3tYrIsLGkJkzHF+w9F2EyxfXGo4UyJc4pFL++FMjnq0HJS69T3M7d//gKrFKwy429bouPescbjecU+Zw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/trim-repeated": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/trim-repeated/-/trim-repeated-1.0.0.tgz", + "integrity": "sha1-42RqLqTokTEr9+rObPsFOAvAHCE=", + "dependencies": { + "escape-string-regexp": "^1.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/trough": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/trough/-/trough-1.0.5.tgz", + "integrity": "sha512-rvuRbTarPXmMb79SmzEp8aqXNKcK+y0XaB298IXueQ8I2PsrATcPBCSPyK/dDNa2iWOhKlfNnOjdAOTBU/nkFA==" + }, + "node_modules/true-case-path": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/true-case-path/-/true-case-path-2.2.1.tgz", + "integrity": "sha512-0z3j8R7MCjy10kc/g+qg7Ln3alJTodw9aDuVWZa3uiWqfuBMKeAeP2ocWcxoyM3D73yz3Jt/Pu4qPr4wHSdB/Q==" + }, + "node_modules/ts-node": { + "version": "9.1.1", + "resolved": "https://registry.npmjs.org/ts-node/-/ts-node-9.1.1.tgz", + "integrity": "sha512-hPlt7ZACERQGf03M253ytLY3dHbGNGrAq9qIHWUY9XHYl1z7wYngSr3OQ5xmui8o2AaxsONxIzjafLUiWBo1Fg==", + "dependencies": { + "arg": "^4.1.0", + "create-require": "^1.1.0", + "diff": "^4.0.1", + "make-error": "^1.1.1", + "source-map-support": "^0.5.17", + "yn": "3.1.1" + }, + "bin": { + "ts-node": "dist/bin.js", + "ts-node-script": "dist/bin-script.js", + "ts-node-transpile-only": "dist/bin-transpile.js", + "ts-script": "dist/bin-script-deprecated.js" + }, + "engines": { + "node": ">=10.0.0" + }, + "peerDependencies": { + "typescript": ">=2.7" + } + }, + "node_modules/tsconfig-paths": { + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/tsconfig-paths/-/tsconfig-paths-3.14.1.tgz", + "integrity": "sha512-fxDhWnFSLt3VuTwtvJt5fpwxBHg5AdKWMsgcPOOIilyjymcYVZoCQF8fvFRezCNfblEXmi+PcM1eYHeOAgXCOQ==", + "dependencies": { + "@types/json5": "^0.0.29", + "json5": "^1.0.1", + "minimist": "^1.2.6", + "strip-bom": "^3.0.0" + } + }, + "node_modules/tsconfig-paths/node_modules/json5": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "dependencies": { + "minimist": "^1.2.0" + }, + "bin": { + "json5": "lib/cli.js" + } + }, + "node_modules/tslib": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.3.1.tgz", + "integrity": "sha512-77EbyPPpMz+FRFRuAFlWMtmgUWGe9UOG2Z25NqCwiIjRhOf5iKGuzSe5P2w1laq+FkRy4p+PCuVkJSGkzTEKVw==" + }, + "node_modules/tsutils": { + "version": "3.21.0", + "resolved": "https://registry.npmjs.org/tsutils/-/tsutils-3.21.0.tgz", + "integrity": "sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA==", + "dependencies": { + "tslib": "^1.8.1" + }, + "engines": { + "node": ">= 6" + }, + "peerDependencies": { + "typescript": ">=2.8.0 || >= 3.2.0-dev || >= 3.3.0-dev || >= 3.4.0-dev || >= 3.5.0-dev || >= 3.6.0-dev || >= 3.6.0-beta || >= 3.7.0-dev || >= 3.7.0-beta" + } + }, + "node_modules/tsutils/node_modules/tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + }, + "node_modules/tunnel-agent": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/tunnel-agent/-/tunnel-agent-0.6.0.tgz", + "integrity": "sha1-J6XeoGs2sEoKmWZ3SykIaPD8QP0=", + "dependencies": { + "safe-buffer": "^5.0.1" + }, + "engines": { + "node": "*" + } + }, + "node_modules/tweetnacl": { + "version": "0.14.5", + "resolved": "https://registry.npmmirror.com/tweetnacl/-/tweetnacl-0.14.5.tgz", + "integrity": "sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==" + }, + "node_modules/type": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/type/-/type-1.2.0.tgz", + "integrity": "sha512-+5nt5AAniqsCnu2cEQQdpzCAh33kVx8n0VoFidKpB1dVVLAN/F+bgVOqOJqOnEnrhp222clB5p3vUlD+1QAnfg==" + }, + "node_modules/type-check": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", + "integrity": "sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew==", + "dependencies": { + "prelude-ls": "^1.2.1" + }, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/type-fest": { + "version": "0.20.2", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", + "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", + "engines": { + "node": ">=10" + } + }, + "node_modules/type-is": { + "version": "1.6.18", + "resolved": "https://registry.npmjs.org/type-is/-/type-is-1.6.18.tgz", + "integrity": "sha512-TkRKr9sUTxEH8MdfuCSP7VizJyzRNMjj2J2do2Jr3Kym598JVdEksuzPQCnlFPW4ky9Q+iA+ma9BGm06XQBy8g==", + "dependencies": { + "media-typer": "0.3.0", + "mime-types": "~2.1.24" + }, + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/type-of": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/type-of/-/type-of-2.0.1.tgz", + "integrity": "sha1-5yoXQYllaOn2KDeNgW1pEvfyOXI=" + }, + "node_modules/typedarray": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/typedarray/-/typedarray-0.0.6.tgz", + "integrity": "sha1-hnrHTjhkGHsdPUfZlqeOxciDB3c=" + }, + "node_modules/typedarray-to-buffer": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz", + "integrity": "sha512-zdu8XMNEDepKKR+XYOXAVPtWui0ly0NtohUscw+UmaHiAWT8hrV1rr//H6V+0DvJ3OQ19S979M0laLfX8rm82Q==", + "dependencies": { + "is-typedarray": "^1.0.0" + } + }, + "node_modules/ua-parser-js": { + "version": "0.7.31", + "resolved": "https://registry.npmjs.org/ua-parser-js/-/ua-parser-js-0.7.31.tgz", + "integrity": "sha512-qLK/Xe9E2uzmYI3qLeOmI0tEOt+TBBQyUIAh4aAgU05FVYzeZrKUdkAZfBNVGRaHVgV0TDkdEngJSw/SyQchkQ==", + "engines": { + "node": "*" + } + }, + "node_modules/unbox-primitive": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/unbox-primitive/-/unbox-primitive-1.0.2.tgz", + "integrity": "sha512-61pPlCD9h51VoreyJ0BReideM3MDKMKnh6+V9L08331ipq6Q8OFXZYiqP6n/tbHx4s5I9uRhcye6BrbkizkBDw==", + "dependencies": { + "call-bind": "^1.0.2", + "has-bigints": "^1.0.2", + "has-symbols": "^1.0.3", + "which-boxed-primitive": "^1.0.2" + } + }, + "node_modules/unc-path-regex": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/unc-path-regex/-/unc-path-regex-0.1.2.tgz", + "integrity": "sha1-5z3T17DXxe2G+6xrCufYxqadUPo=", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/underscore.string": { + "version": "3.3.6", + "resolved": "https://registry.npmmirror.com/underscore.string/-/underscore.string-3.3.6.tgz", + "integrity": "sha512-VoC83HWXmCrF6rgkyxS9GHv8W9Q5nhMKho+OadDJGzL2oDYbYEppBaCMH6pFlwLeqj2QS+hhkw2kpXkSdD1JxQ==", + "dependencies": { + "sprintf-js": "^1.1.1", + "util-deprecate": "^1.0.2" + }, + "engines": { + "node": "*" + } + }, + "node_modules/underscore.string/node_modules/sprintf-js": { + "version": "1.1.2", + "resolved": "https://registry.npmmirror.com/sprintf-js/-/sprintf-js-1.1.2.tgz", + "integrity": "sha512-VE0SOVEHCk7Qc8ulkWw3ntAzXuqf7S2lvwQaDLRnUeIEaKNQJzV6BwmLKhOqT61aGhfUMrXeaBk+oDGCzvhcug==" + }, + "node_modules/unherit": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/unherit/-/unherit-1.1.3.tgz", + "integrity": "sha512-Ft16BJcnapDKp0+J/rqFC3Rrk6Y/Ng4nzsC028k2jdDII/rdZ7Wd3pPT/6+vIIxRagwRc9K0IUX0Ra4fKvw+WQ==", + "dependencies": { + "inherits": "^2.0.0", + "xtend": "^4.0.0" + } + }, + "node_modules/unicode-canonical-property-names-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz", + "integrity": "sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/unicode-match-property-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz", + "integrity": "sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q==", + "dependencies": { + "unicode-canonical-property-names-ecmascript": "^2.0.0", + "unicode-property-aliases-ecmascript": "^2.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/unicode-match-property-value-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.0.0.tgz", + "integrity": "sha512-7Yhkc0Ye+t4PNYzOGKedDhXbYIBe1XEQYQxOPyhcXNMJ0WCABqqj6ckydd6pWRZTHV4GuCPKdBAUiMc60tsKVw==", + "engines": { + "node": ">=4" + } + }, + "node_modules/unicode-property-aliases-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.0.0.tgz", + "integrity": "sha512-5Zfuy9q/DFr4tfO7ZPeVXb1aPoeQSdeFMLpYuFebehDAhbuevLs5yxSZmIFN1tP5F9Wl4IpJrYojg85/zgyZHQ==", + "engines": { + "node": ">=4" + } + }, + "node_modules/unified": { + "version": "9.2.2", + "resolved": "https://registry.npmmirror.com/unified/-/unified-9.2.2.tgz", + "integrity": "sha512-Sg7j110mtefBD+qunSLO1lqOEKdrwBFBrR6Qd8f4uwkhWNlbkaqwHse6e7QvD3AP/MNoJdEDLaf8OxYyoWgorQ==", + "dependencies": { + "bail": "^1.0.0", + "extend": "^3.0.0", + "is-buffer": "^2.0.0", + "is-plain-obj": "^2.0.0", + "trough": "^1.0.0", + "vfile": "^4.0.0" + } + }, + "node_modules/unified/node_modules/is-plain-obj": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-2.1.0.tgz", + "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", + "engines": { + "node": ">=8" + } + }, + "node_modules/unique-string": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unique-string/-/unique-string-2.0.0.tgz", + "integrity": "sha512-uNaeirEPvpZWSgzwsPGtU2zVSTrn/8L5q/IexZmH0eH6SA73CmAA5U4GwORTxQAZs95TAXLNqeLoPPNO5gZfWg==", + "dependencies": { + "crypto-random-string": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/unist-builder": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-builder/-/unist-builder-2.0.3.tgz", + "integrity": "sha512-f98yt5pnlMWlzP539tPc4grGMsFaQQlP/vM396b00jngsiINumNmsY8rkXjfoi1c6QaM8nQ3vaGDuoKWbe/1Uw==" + }, + "node_modules/unist-util-generated": { + "version": "1.1.6", + "resolved": "https://registry.npmmirror.com/unist-util-generated/-/unist-util-generated-1.1.6.tgz", + "integrity": "sha512-cln2Mm1/CZzN5ttGK7vkoGw+RZ8VcUH6BtGbq98DDtRGquAAOXig1mrBQYelOwMXYS8rK+vZDyyojSjp7JX+Lg==" + }, + "node_modules/unist-util-is": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/unist-util-is/-/unist-util-is-4.1.0.tgz", + "integrity": "sha512-ZOQSsnce92GrxSqlnEEseX0gi7GH9zTJZ0p9dtu87WRb/37mMPO2Ilx1s/t9vBHrFhbgweUwb+t7cIn5dxPhZg==" + }, + "node_modules/unist-util-modify-children": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/unist-util-modify-children/-/unist-util-modify-children-2.0.0.tgz", + "integrity": "sha512-HGrj7JQo9DwZt8XFsX8UD4gGqOsIlCih9opG6Y+N11XqkBGKzHo8cvDi+MfQQgiZ7zXRUiQREYHhjOBHERTMdg==", + "dependencies": { + "array-iterate": "^1.0.0" + } + }, + "node_modules/unist-util-position": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/unist-util-position/-/unist-util-position-3.1.0.tgz", + "integrity": "sha512-w+PkwCbYSFw8vpgWD0v7zRCl1FpY3fjDSQ3/N/wNd9Ffa4gPi8+4keqt99N3XW6F99t/mUzp2xAhNmfKWp95QA==" + }, + "node_modules/unist-util-remove-position": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/unist-util-remove-position/-/unist-util-remove-position-3.0.0.tgz", + "integrity": "sha512-17kIOuolVuK16LMb9KyMJlqdfCtlfQY5FjY3Sdo9iC7F5wqdXhNjMq0PBvMpkVNNnAmHxXssUW+rZ9T2zbP0Rg==", + "dependencies": { + "unist-util-visit": "^2.0.0" + } + }, + "node_modules/unist-util-select": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/unist-util-select/-/unist-util-select-3.0.4.tgz", + "integrity": "sha512-xf1zCu4okgPqGLdhCDpRnjwBNyv3EqjiXRUbz2SdK1+qnLMB7uXXajfzuBvvbHoQ+JLyp4AEbFCGndmc6S72sw==", + "dependencies": { + "css-selector-parser": "^1.0.0", + "not": "^0.1.0", + "nth-check": "^2.0.0", + "unist-util-is": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "node_modules/unist-util-stringify-position": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-util-stringify-position/-/unist-util-stringify-position-2.0.3.tgz", + "integrity": "sha512-3faScn5I+hy9VleOq/qNbAd6pAx7iH5jYBMS9I1HgQVijz/4mv5Bvw5iw1sC/90CODiKo81G/ps8AJrISn687g==", + "dependencies": { + "@types/unist": "^2.0.2" + } + }, + "node_modules/unist-util-visit": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-util-visit/-/unist-util-visit-2.0.3.tgz", + "integrity": "sha512-iJ4/RczbJMkD0712mGktuGpm/U4By4FfDonL7N/9tATGIF4imikjOuagyMY53tnZq3NP6BcmlrHhEKAfGWjh7Q==", + "dependencies": { + "@types/unist": "^2.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit-parents": "^3.0.0" + } + }, + "node_modules/unist-util-visit-children": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/unist-util-visit-children/-/unist-util-visit-children-1.1.4.tgz", + "integrity": "sha512-sA/nXwYRCQVRwZU2/tQWUqJ9JSFM1X3x7JIOsIgSzrFHcfVt6NkzDtKzyxg2cZWkCwGF9CO8x4QNZRJRMK8FeQ==" + }, + "node_modules/unist-util-visit-parents": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/unist-util-visit-parents/-/unist-util-visit-parents-3.1.1.tgz", + "integrity": "sha512-1KROIZWo6bcMrZEwiH2UrXDyalAa0uqzWCxCJj6lPOvTve2WkfgCytoDTPaMnodXh1WrXOq0haVYHj99ynJlsg==", + "dependencies": { + "@types/unist": "^2.0.0", + "unist-util-is": "^4.0.0" + } + }, + "node_modules/universalify": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/universalify/-/universalify-2.0.0.tgz", + "integrity": "sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ==", + "engines": { + "node": ">= 10.0.0" + } + }, + "node_modules/unixify": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/unixify/-/unixify-1.0.0.tgz", + "integrity": "sha1-OmQcjC/7zk2mg6XHDwOkYpQMIJA=", + "dependencies": { + "normalize-path": "^2.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/unixify/node_modules/normalize-path": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-2.1.1.tgz", + "integrity": "sha1-GrKLVW4Zg2Oowab35vogE3/mrtk=", + "dependencies": { + "remove-trailing-separator": "^1.0.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/unpipe": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/unpipe/-/unpipe-1.0.0.tgz", + "integrity": "sha1-sr9O6FFKrmFltIF4KdIbLvSZBOw=", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/unquote": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/unquote/-/unquote-1.1.1.tgz", + "integrity": "sha1-j97XMk7G6IoP+LkF58CYzcCG1UQ=" + }, + "node_modules/update-notifier": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/update-notifier/-/update-notifier-5.1.0.tgz", + "integrity": "sha512-ItnICHbeMh9GqUy31hFPrD1kcuZ3rpxDZbf4KUDavXwS0bW5m7SLbDQpGX3UYr072cbrF5hFUs3r5tUsPwjfHw==", + "dependencies": { + "boxen": "^5.0.0", + "chalk": "^4.1.0", + "configstore": "^5.0.1", + "has-yarn": "^2.1.0", + "import-lazy": "^2.1.0", + "is-ci": "^2.0.0", + "is-installed-globally": "^0.4.0", + "is-npm": "^5.0.0", + "is-yarn-global": "^0.3.0", + "latest-version": "^5.1.0", + "pupa": "^2.1.1", + "semver": "^7.3.4", + "semver-diff": "^3.1.1", + "xdg-basedir": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/upper-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/upper-case/-/upper-case-2.0.2.tgz", + "integrity": "sha512-KgdgDGJt2TpuwBUIjgG6lzw2GWFRCW9Qkfkiv0DxqHHLYJHmtmdUIKcZd8rHgFSjopVTlw6ggzCm1b8MFQwikg==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/upper-case-first": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/upper-case-first/-/upper-case-first-2.0.2.tgz", + "integrity": "sha512-514ppYHBaKwfJRK/pNC6c/OxfGa0obSnAl106u97Ed0I625Nin96KAjttZF6ZL3e1XLtphxnqrOi9iWgm+u+bg==", + "dependencies": { + "tslib": "^2.0.3" + } + }, + "node_modules/uri-js": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/uri-js/-/uri-js-4.4.1.tgz", + "integrity": "sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg==", + "dependencies": { + "punycode": "^2.1.0" + } + }, + "node_modules/urix": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/urix/-/urix-0.1.0.tgz", + "integrity": "sha512-Am1ousAhSLBeB9cG/7k7r2R0zj50uDRlZHPGbazid5s9rlF1F/QKYObEKSIunSjIOkJZqwRRLpvewjEkM7pSqg==", + "deprecated": "Please see https://github.com/lydell/urix#deprecated" + }, + "node_modules/url-loader": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/url-loader/-/url-loader-4.1.1.tgz", + "integrity": "sha512-3BTV812+AVHHOJQO8O5MkWgZ5aosP7GnROJwvzLS9hWDj00lZ6Z0wNak423Lp9PBZN05N+Jk/N5Si8jRAlGyWA==", + "dependencies": { + "loader-utils": "^2.0.0", + "mime-types": "^2.1.27", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= 10.13.0" + }, + "peerDependencies": { + "file-loader": "*", + "webpack": "^4.0.0 || ^5.0.0" + }, + "peerDependenciesMeta": { + "file-loader": { + "optional": true + } + } + }, + "node_modules/url-loader/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/url-parse-lax": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/url-parse-lax/-/url-parse-lax-3.0.0.tgz", + "integrity": "sha1-FrXK/Afb42dsGxmZF3gj1lA6yww=", + "dependencies": { + "prepend-http": "^2.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/use-composed-ref": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/use-composed-ref/-/use-composed-ref-1.3.0.tgz", + "integrity": "sha512-GLMG0Jc/jiKov/3Ulid1wbv3r54K9HlMW29IWcDFPEqFkSO2nS0MuefWgMJpeHQ9YJeXDL3ZUF+P3jdXlZX/cQ==", + "peerDependencies": { + "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + } + }, + "node_modules/use-isomorphic-layout-effect": { + "version": "1.1.2", + "resolved": "https://registry.npmmirror.com/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.1.2.tgz", + "integrity": "sha512-49L8yCO3iGT/ZF9QttjwLF/ZD9Iwto5LnH5LmEdk/6cFmXddqi2ulF0edxTwjj+7mqvpVVGQWvbXZdn32wRSHA==", + "peerDependencies": { + "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, + "node_modules/use-latest": { + "version": "1.2.1", + "resolved": "https://registry.npmmirror.com/use-latest/-/use-latest-1.2.1.tgz", + "integrity": "sha512-xA+AVm/Wlg3e2P/JiItTziwS7FK92LWrDB0p+hgXloIMuVCeJJ8v6f0eeHyPZaJrM+usM1FkFfbNCrJGs8A/zw==", + "dependencies": { + "use-isomorphic-layout-effect": "^1.1.1" + }, + "peerDependencies": { + "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, + "node_modules/utif": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/utif/-/utif-2.0.1.tgz", + "integrity": "sha512-Z/S1fNKCicQTf375lIP9G8Sa1H/phcysstNrrSdZKj1f9g58J4NMgb5IgiEZN9/nLMPDwF0W7hdOe9Qq2IYoLg==", + "dependencies": { + "pako": "^1.0.5" + } + }, + "node_modules/util-deprecate": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", + "integrity": "sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8=" + }, + "node_modules/util.promisify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/util.promisify/-/util.promisify-1.0.1.tgz", + "integrity": "sha512-g9JpC/3He3bm38zsLupWryXHoEcS22YHthuPQSJdMy6KNrzIRzWqcsHzD/WUnqe45whVou4VIsPew37DoXWNrA==", + "dependencies": { + "define-properties": "^1.1.3", + "es-abstract": "^1.17.2", + "has-symbols": "^1.0.1", + "object.getownpropertydescriptors": "^2.1.0" + } + }, + "node_modules/utila": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/utila/-/utila-0.4.0.tgz", + "integrity": "sha1-ihagXURWV6Oupe7MWxKk+lN5dyw=" + }, + "node_modules/utility-types": { + "version": "3.10.0", + "resolved": "https://registry.npmjs.org/utility-types/-/utility-types-3.10.0.tgz", + "integrity": "sha512-O11mqxmi7wMKCo6HKFt5AhO4BwY3VV68YU07tgxfz8zJTIxr4BpsezN49Ffwy9j3ZpwwJp4fkRwjRzq3uWE6Rg==", + "engines": { + "node": ">= 4" + } + }, + "node_modules/utils-merge": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/utils-merge/-/utils-merge-1.0.1.tgz", + "integrity": "sha1-n5VxD1CiZ5R7LMwSR0HBAoQn5xM=", + "engines": { + "node": ">= 0.4.0" + } + }, + "node_modules/uuid": { + "version": "8.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", + "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", + "bin": { + "uuid": "dist/bin/uuid" + } + }, + "node_modules/v8-compile-cache": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/v8-compile-cache/-/v8-compile-cache-2.3.0.tgz", + "integrity": "sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA==" + }, + "node_modules/valid-url": { + "version": "1.0.9", + "resolved": "https://registry.npmjs.org/valid-url/-/valid-url-1.0.9.tgz", + "integrity": "sha1-HBRHm0DxOXp1eC8RXkCGRHQzogA=" + }, + "node_modules/validate-npm-package-license": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz", + "integrity": "sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew==", + "dependencies": { + "spdx-correct": "^3.0.0", + "spdx-expression-parse": "^3.0.0" + } + }, + "node_modules/value-or-promise": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.11.tgz", + "integrity": "sha512-41BrgH+dIbCFXClcSapVs5M6GkENd3gQOJpEfPDNa71LsUGMXDL0jMWpI/Rh7WhX+Aalfz2TTS3Zt5pUsbnhLg==", + "engines": { + "node": ">=12" + } + }, + "node_modules/vary": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/vary/-/vary-1.1.2.tgz", + "integrity": "sha1-IpnwLG3tMNSllhsLn3RSShj2NPw=", + "engines": { + "node": ">= 0.8" + } + }, + "node_modules/verror": { + "version": "1.10.0", + "resolved": "https://registry.npmmirror.com/verror/-/verror-1.10.0.tgz", + "integrity": "sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==", + "engines": [ + "node >=0.6.0" + ], + "dependencies": { + "assert-plus": "^1.0.0", + "core-util-is": "1.0.2", + "extsprintf": "^1.2.0" + } + }, + "node_modules/verror/node_modules/core-util-is": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/core-util-is/-/core-util-is-1.0.2.tgz", + "integrity": "sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==" + }, + "node_modules/vfile": { + "version": "4.2.1", + "resolved": "https://registry.npmmirror.com/vfile/-/vfile-4.2.1.tgz", + "integrity": "sha512-O6AE4OskCG5S1emQ/4gl8zK586RqA3srz3nfK/Viy0UPToBc5Trp9BVFb1u0CjsKrAWwnpr4ifM/KBXPWwJbCA==", + "dependencies": { + "@types/unist": "^2.0.0", + "is-buffer": "^2.0.0", + "unist-util-stringify-position": "^2.0.0", + "vfile-message": "^2.0.0" + } + }, + "node_modules/vfile-location": { + "version": "3.2.0", + "resolved": "https://registry.npmmirror.com/vfile-location/-/vfile-location-3.2.0.tgz", + "integrity": "sha512-aLEIZKv/oxuCDZ8lkJGhuhztf/BW4M+iHdCwglA/eWc+vtuRFJj8EtgceYFX4LRjOhCAAiNHsKGssC6onJ+jbA==" + }, + "node_modules/vfile-message": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/vfile-message/-/vfile-message-2.0.4.tgz", + "integrity": "sha512-DjssxRGkMvifUOJre00juHoP9DPWuzjxKuMDrhNbk2TdaYYBNMStsNhEOt3idrtI12VQYM/1+iM0KOzXi4pxwQ==", + "dependencies": { + "@types/unist": "^2.0.0", + "unist-util-stringify-position": "^2.0.0" + } + }, + "node_modules/warning": { + "version": "4.0.3", + "resolved": "https://registry.npmmirror.com/warning/-/warning-4.0.3.tgz", + "integrity": "sha512-rpJyN222KWIvHJ/F53XSZv0Zl/accqHR8et1kpaMTD/fLCRxtV8iX8czMzY7sVZupTI3zcUTg8eycS2kNF9l6w==", + "dependencies": { + "loose-envify": "^1.0.0" + } + }, + "node_modules/watchpack": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.3.1.tgz", + "integrity": "sha512-x0t0JuydIo8qCNctdDrn1OzH/qDzk2+rdCOC3YzumZ42fiMqmQ7T3xQurykYMhYfHaPHTp4ZxAx2NfUo1K6QaA==", + "dependencies": { + "glob-to-regexp": "^0.4.1", + "graceful-fs": "^4.1.2" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/weak-lru-cache": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/weak-lru-cache/-/weak-lru-cache-1.2.2.tgz", + "integrity": "sha512-DEAoo25RfSYMuTGc9vPJzZcZullwIqRDSI9LOy+fkCJPi6hykCnfKaXTuPBDuXAUcqHXyOgFtHNp/kB2FjYHbw==" + }, + "node_modules/web-namespaces": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/web-namespaces/-/web-namespaces-1.1.4.tgz", + "integrity": "sha512-wYxSGajtmoP4WxfejAPIr4l0fVh+jeMXZb08wNc0tMg6xsfZXj3cECqIK0G7ZAqUq0PP8WlMDtaOGVBTAWztNw==" + }, + "node_modules/webidl-conversions": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", + "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" + }, + "node_modules/webpack": { + "version": "5.72.1", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.72.1.tgz", + "integrity": "sha512-dXG5zXCLspQR4krZVR6QgajnZOjW2K/djHvdcRaDQvsjV9z9vaW6+ja5dZOYbqBBjF6kGXka/2ZyxNdc+8Jung==", + "dependencies": { + "@types/eslint-scope": "^3.7.3", + "@types/estree": "^0.0.51", + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/wasm-edit": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1", + "acorn": "^8.4.1", + "acorn-import-assertions": "^1.7.6", + "browserslist": "^4.14.5", + "chrome-trace-event": "^1.0.2", + "enhanced-resolve": "^5.9.3", + "es-module-lexer": "^0.9.0", + "eslint-scope": "5.1.1", + "events": "^3.2.0", + "glob-to-regexp": "^0.4.1", + "graceful-fs": "^4.2.9", + "json-parse-even-better-errors": "^2.3.1", + "loader-runner": "^4.2.0", + "mime-types": "^2.1.27", + "neo-async": "^2.6.2", + "schema-utils": "^3.1.0", + "tapable": "^2.1.1", + "terser-webpack-plugin": "^5.1.3", + "watchpack": "^2.3.1", + "webpack-sources": "^3.2.3" + }, + "bin": { + "webpack": "bin/webpack.js" + }, + "engines": { + "node": ">=10.13.0" + }, + "peerDependenciesMeta": { + "webpack-cli": { + "optional": true + } + } + }, + "node_modules/webpack-assets-manifest": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/webpack-assets-manifest/-/webpack-assets-manifest-5.1.0.tgz", + "integrity": "sha512-kPuTMEjBrqZQVJ5M6yXNBCEdFbQQn7p+loNXt8NOeDFaAbsNFWqqwR0YL1mfG5LbwhK5FLXWXpuK3GuIIZ46rg==", + "dependencies": { + "chalk": "^4.0", + "deepmerge": "^4.0", + "lockfile": "^1.0", + "lodash.get": "^4.0", + "lodash.has": "^4.0", + "schema-utils": "^3.0", + "tapable": "^2.0" + }, + "engines": { + "node": ">=10.13.0" + }, + "peerDependencies": { + "webpack": "^5.2.0" + } + }, + "node_modules/webpack-assets-manifest/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/webpack-dev-middleware": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/webpack-dev-middleware/-/webpack-dev-middleware-4.3.0.tgz", + "integrity": "sha512-PjwyVY95/bhBh6VUqt6z4THplYcsvQ8YNNBTBM873xLVmw8FLeALn0qurHbs9EmcfhzQis/eoqypSnZeuUz26w==", + "dependencies": { + "colorette": "^1.2.2", + "mem": "^8.1.1", + "memfs": "^3.2.2", + "mime-types": "^2.1.30", + "range-parser": "^1.2.1", + "schema-utils": "^3.0.0" + }, + "engines": { + "node": ">= v10.23.3" + }, + "peerDependencies": { + "webpack": "^4.0.0 || ^5.0.0" + } + }, + "node_modules/webpack-dev-middleware/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/webpack-merge": { + "version": "5.8.0", + "resolved": "https://registry.npmjs.org/webpack-merge/-/webpack-merge-5.8.0.tgz", + "integrity": "sha512-/SaI7xY0831XwP6kzuwhKWVKDP9t1QY1h65lAFLbZqMPIuYcD9QAW4u9STIbU9kaJbPBB/geU/gLr1wDjOhQ+Q==", + "dependencies": { + "clone-deep": "^4.0.1", + "wildcard": "^2.0.0" + }, + "engines": { + "node": ">=10.0.0" + } + }, + "node_modules/webpack-sources": { + "version": "1.4.3", + "resolved": "https://registry.npmjs.org/webpack-sources/-/webpack-sources-1.4.3.tgz", + "integrity": "sha512-lgTS3Xhv1lCOKo7SA5TjKXMjpSM4sBjNV5+q2bqesbSPs5FjGmU6jjtBSkX9b4qW87vDIsCIlUPOEhbZrMdjeQ==", + "dependencies": { + "source-list-map": "^2.0.0", + "source-map": "~0.6.1" + } + }, + "node_modules/webpack-sources/node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/webpack-stats-plugin": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/webpack-stats-plugin/-/webpack-stats-plugin-1.0.3.tgz", + "integrity": "sha512-tV/SQHl6lKfBahJcNDmz8JG1rpWPB9NEDQSMIoL74oVAotdxYljpgIsgLzgc1N9QrtA9KEA0moJVwQtNZv2aDA==" + }, + "node_modules/webpack-virtual-modules": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/webpack-virtual-modules/-/webpack-virtual-modules-0.3.2.tgz", + "integrity": "sha512-RXQXioY6MhzM4CNQwmBwKXYgBs6ulaiQ8bkNQEl2J6Z+V+s7lgl/wGvaI/I0dLnYKB8cKsxQc17QOAVIphPLDw==", + "dependencies": { + "debug": "^3.0.0" + } + }, + "node_modules/webpack/node_modules/acorn": { + "version": "8.7.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.7.1.tgz", + "integrity": "sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==", + "bin": { + "acorn": "bin/acorn" + }, + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/webpack/node_modules/schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "dependencies": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + }, + "engines": { + "node": ">= 10.13.0" + } + }, + "node_modules/webpack/node_modules/webpack-sources": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/webpack-sources/-/webpack-sources-3.2.3.tgz", + "integrity": "sha512-/DyMEOrDgLKKIG0fmvtz+4dUX/3Ghozwgm6iPp8KRhvn+eQf9+Q7GWxVNMk3+uCPWfdXYC4ExGBckIXdFEfH1w==", + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/whatwg-url": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", + "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", + "dependencies": { + "tr46": "~0.0.3", + "webidl-conversions": "^3.0.0" + } + }, + "node_modules/which": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/which/-/which-1.3.1.tgz", + "integrity": "sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ==", + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "which": "bin/which" + } + }, + "node_modules/which-boxed-primitive": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz", + "integrity": "sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg==", + "dependencies": { + "is-bigint": "^1.0.1", + "is-boolean-object": "^1.1.0", + "is-number-object": "^1.0.4", + "is-string": "^1.0.5", + "is-symbol": "^1.0.3" + } + }, + "node_modules/which-module": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/which-module/-/which-module-2.0.0.tgz", + "integrity": "sha1-2e8H3Od7mQK4o6j6SzHD4/fm6Ho=" + }, + "node_modules/wide-align": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.5.tgz", + "integrity": "sha512-eDMORYaPNZ4sQIuuYPDHdQvf4gyCF9rEEV/yPxGfwPkRodwEgiMUUXTx/dex+Me0wxx53S+NgUHaP7y3MGlDmg==", + "dependencies": { + "string-width": "^1.0.2 || 2 || 3 || 4" + } + }, + "node_modules/widest-line": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/widest-line/-/widest-line-3.1.0.tgz", + "integrity": "sha512-NsmoXalsWVDMGupxZ5R08ka9flZjjiLvHVAWYOKtiKM8ujtZWr9cRffak+uSE48+Ob8ObalXpwyeUiyDD6QFgg==", + "dependencies": { + "string-width": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/wildcard": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/wildcard/-/wildcard-2.0.0.tgz", + "integrity": "sha512-JcKqAHLPxcdb9KM49dufGXn2x3ssnfjbcaQdLlfZsL9rH9wgDQjUtDxbo8NE0F6SFvydeu1VhZe7hZuHsB2/pw==" + }, + "node_modules/word-wrap": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/word-wrap/-/word-wrap-1.2.3.tgz", + "integrity": "sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/workbox-background-sync": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-background-sync/-/workbox-background-sync-4.3.1.tgz", + "integrity": "sha512-1uFkvU8JXi7L7fCHVBEEnc3asPpiAL33kO495UMcD5+arew9IbKW2rV5lpzhoWcm/qhGB89YfO4PmB/0hQwPRg==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-broadcast-update": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-broadcast-update/-/workbox-broadcast-update-4.3.1.tgz", + "integrity": "sha512-MTSfgzIljpKLTBPROo4IpKjESD86pPFlZwlvVG32Kb70hW+aob4Jxpblud8EhNb1/L5m43DUM4q7C+W6eQMMbA==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-build": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-build/-/workbox-build-4.3.1.tgz", + "integrity": "sha512-UHdwrN3FrDvicM3AqJS/J07X0KXj67R8Cg0waq1MKEOqzo89ap6zh6LmaLnRAjpB+bDIz+7OlPye9iii9KBnxw==", + "dependencies": { + "@babel/runtime": "^7.3.4", + "@hapi/joi": "^15.0.0", + "common-tags": "^1.8.0", + "fs-extra": "^4.0.2", + "glob": "^7.1.3", + "lodash.template": "^4.4.0", + "pretty-bytes": "^5.1.0", + "stringify-object": "^3.3.0", + "strip-comments": "^1.0.2", + "workbox-background-sync": "^4.3.1", + "workbox-broadcast-update": "^4.3.1", + "workbox-cacheable-response": "^4.3.1", + "workbox-core": "^4.3.1", + "workbox-expiration": "^4.3.1", + "workbox-google-analytics": "^4.3.1", + "workbox-navigation-preload": "^4.3.1", + "workbox-precaching": "^4.3.1", + "workbox-range-requests": "^4.3.1", + "workbox-routing": "^4.3.1", + "workbox-strategies": "^4.3.1", + "workbox-streams": "^4.3.1", + "workbox-sw": "^4.3.1", + "workbox-window": "^4.3.1" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/workbox-build/node_modules/fs-extra": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-4.0.3.tgz", + "integrity": "sha512-q6rbdDd1o2mAnQreO7YADIxf/Whx4AHBiRf6d+/cVT8h44ss+lHgxf1FemcqDnQt9X3ct4McHr+JMGlYSsK7Cg==", + "dependencies": { + "graceful-fs": "^4.1.2", + "jsonfile": "^4.0.0", + "universalify": "^0.1.0" + } + }, + "node_modules/workbox-build/node_modules/jsonfile": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-4.0.0.tgz", + "integrity": "sha1-h3Gq4HmbZAdrdmQPygWPnBDjPss=", + "optionalDependencies": { + "graceful-fs": "^4.1.6" + } + }, + "node_modules/workbox-build/node_modules/universalify": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/universalify/-/universalify-0.1.2.tgz", + "integrity": "sha512-rBJeI5CXAlmy1pV+617WB9J63U6XcazHHF2f2dbJix4XzpUF0RS3Zbj0FGIOCAva5P/d/GBOYaACQ1w+0azUkg==", + "engines": { + "node": ">= 4.0.0" + } + }, + "node_modules/workbox-cacheable-response": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-cacheable-response/-/workbox-cacheable-response-4.3.1.tgz", + "integrity": "sha512-Rp5qlzm6z8IOvnQNkCdO9qrDgDpoPNguovs0H8C+wswLuPgSzSp9p2afb5maUt9R1uTIwOXrVQMmPfPypv+npw==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-core": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-core/-/workbox-core-4.3.1.tgz", + "integrity": "sha512-I3C9jlLmMKPxAC1t0ExCq+QoAMd0vAAHULEgRZ7kieCdUd919n53WC0AfvokHNwqRhGn+tIIj7vcb5duCjs2Kg==" + }, + "node_modules/workbox-expiration": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-expiration/-/workbox-expiration-4.3.1.tgz", + "integrity": "sha512-vsJLhgQsQouv9m0rpbXubT5jw0jMQdjpkum0uT+d9tTwhXcEZks7qLfQ9dGSaufTD2eimxbUOJfWLbNQpIDMPw==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-google-analytics": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-google-analytics/-/workbox-google-analytics-4.3.1.tgz", + "integrity": "sha512-xzCjAoKuOb55CBSwQrbyWBKqp35yg1vw9ohIlU2wTy06ZrYfJ8rKochb1MSGlnoBfXGWss3UPzxR5QL5guIFdg==", + "dependencies": { + "workbox-background-sync": "^4.3.1", + "workbox-core": "^4.3.1", + "workbox-routing": "^4.3.1", + "workbox-strategies": "^4.3.1" + } + }, + "node_modules/workbox-navigation-preload": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-navigation-preload/-/workbox-navigation-preload-4.3.1.tgz", + "integrity": "sha512-K076n3oFHYp16/C+F8CwrRqD25GitA6Rkd6+qAmLmMv1QHPI2jfDwYqrytOfKfYq42bYtW8Pr21ejZX7GvALOw==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-precaching": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-precaching/-/workbox-precaching-4.3.1.tgz", + "integrity": "sha512-piSg/2csPoIi/vPpp48t1q5JLYjMkmg5gsXBQkh/QYapCdVwwmKlU9mHdmy52KsDGIjVaqEUMFvEzn2LRaigqQ==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-range-requests": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-range-requests/-/workbox-range-requests-4.3.1.tgz", + "integrity": "sha512-S+HhL9+iTFypJZ/yQSl/x2Bf5pWnbXdd3j57xnb0V60FW1LVn9LRZkPtneODklzYuFZv7qK6riZ5BNyc0R0jZA==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-routing": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-routing/-/workbox-routing-4.3.1.tgz", + "integrity": "sha512-FkbtrODA4Imsi0p7TW9u9MXuQ5P4pVs1sWHK4dJMMChVROsbEltuE79fBoIk/BCztvOJ7yUpErMKa4z3uQLX+g==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-strategies": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-strategies/-/workbox-strategies-4.3.1.tgz", + "integrity": "sha512-F/+E57BmVG8dX6dCCopBlkDvvhg/zj6VDs0PigYwSN23L8hseSRwljrceU2WzTvk/+BSYICsWmRq5qHS2UYzhw==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-streams": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-streams/-/workbox-streams-4.3.1.tgz", + "integrity": "sha512-4Kisis1f/y0ihf4l3u/+ndMkJkIT4/6UOacU3A4BwZSAC9pQ9vSvJpIi/WFGQRH/uPXvuVjF5c2RfIPQFSS2uA==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/workbox-sw": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-sw/-/workbox-sw-4.3.1.tgz", + "integrity": "sha512-0jXdusCL2uC5gM3yYFT6QMBzKfBr2XTk0g5TPAV4y8IZDyVNDyj1a8uSXy3/XrvkVTmQvLN4O5k3JawGReXr9w==" + }, + "node_modules/workbox-window": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-window/-/workbox-window-4.3.1.tgz", + "integrity": "sha512-C5gWKh6I58w3GeSc0wp2Ne+rqVw8qwcmZnQGpjiek8A2wpbxSJb1FdCoQVO+jDJs35bFgo/WETgl1fqgsxN0Hg==", + "dependencies": { + "workbox-core": "^4.3.1" + } + }, + "node_modules/wrap-ansi": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-6.2.0.tgz", + "integrity": "sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA==", + "dependencies": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/wrap-ansi/node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/wrap-ansi/node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/wrap-ansi/node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "node_modules/wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + }, + "node_modules/write-file-atomic": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/write-file-atomic/-/write-file-atomic-3.0.3.tgz", + "integrity": "sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q==", + "dependencies": { + "imurmurhash": "^0.1.4", + "is-typedarray": "^1.0.0", + "signal-exit": "^3.0.2", + "typedarray-to-buffer": "^3.1.5" + } + }, + "node_modules/ws": { + "version": "7.4.5", + "resolved": "https://registry.npmjs.org/ws/-/ws-7.4.5.tgz", + "integrity": "sha512-xzyu3hFvomRfXKH8vOFMU3OguG6oOvhXMo3xsGy3xWExqaM2dxBbVxuD99O7m3ZUFMvvscsZDqxfgMaRr/Nr1g==", + "engines": { + "node": ">=8.3.0" + }, + "peerDependencies": { + "bufferutil": "^4.0.1", + "utf-8-validate": "^5.0.2" + }, + "peerDependenciesMeta": { + "bufferutil": { + "optional": true + }, + "utf-8-validate": { + "optional": true + } + } + }, + "node_modules/xdg-basedir": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/xdg-basedir/-/xdg-basedir-4.0.0.tgz", + "integrity": "sha512-PSNhEJDejZYV7h50BohL09Er9VaIefr2LMAf3OEmpCkjOi34eYyQYAXUTjEQtZJTKcF0E2UKTh+osDLsgNim9Q==", + "engines": { + "node": ">=8" + } + }, + "node_modules/xhr": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/xhr/-/xhr-2.6.0.tgz", + "integrity": "sha512-/eCGLb5rxjx5e3mF1A7s+pLlR6CGyqWN91fv1JgER5mVWg1MZmlhBvy9kjcsOdRk8RrIujotWyJamfyrp+WIcA==", + "dependencies": { + "global": "~4.4.0", + "is-function": "^1.0.1", + "parse-headers": "^2.0.0", + "xtend": "^4.0.0" + } + }, + "node_modules/xml-parse-from-string": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/xml-parse-from-string/-/xml-parse-from-string-1.0.1.tgz", + "integrity": "sha1-qQKekp09vN7RafPG4oI42VpdWig=" + }, + "node_modules/xml2js": { + "version": "0.4.23", + "resolved": "https://registry.npmjs.org/xml2js/-/xml2js-0.4.23.tgz", + "integrity": "sha512-ySPiMjM0+pLDftHgXY4By0uswI3SPKLDw/i3UXbnO8M/p28zqexCUoPmQFrYD+/1BzhGJSs2i1ERWKJAtiLrug==", + "dependencies": { + "sax": ">=0.6.0", + "xmlbuilder": "~11.0.0" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/xmlbuilder": { + "version": "11.0.1", + "resolved": "https://registry.npmjs.org/xmlbuilder/-/xmlbuilder-11.0.1.tgz", + "integrity": "sha512-fDlsI/kFEx7gLvbecc0/ohLG50fugQp8ryHzMTuW9vSa1GJ0XYWKnhsUx7oie3G98+r56aTQIUB4kht42R3JvA==", + "engines": { + "node": ">=4.0" + } + }, + "node_modules/xmlhttprequest-ssl": { + "version": "1.6.3", + "resolved": "https://registry.npmjs.org/xmlhttprequest-ssl/-/xmlhttprequest-ssl-1.6.3.tgz", + "integrity": "sha512-3XfeQE/wNkvrIktn2Kf0869fC0BN6UpydVasGIeSm2B1Llihf7/0UfZM+eCkOw3P7bP4+qPgqhm7ZoxuJtFU0Q==", + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/xss": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/xss/-/xss-1.0.11.tgz", + "integrity": "sha512-EimjrjThZeK2MO7WKR9mN5ZC1CSqivSl55wvUK5EtU6acf0rzEE1pN+9ZDrFXJ82BRp3JL38pPE6S4o/rpp1zQ==", + "dependencies": { + "commander": "^2.20.3", + "cssfilter": "0.0.10" + }, + "bin": { + "xss": "bin/xss" + }, + "engines": { + "node": ">= 0.10.0" + } + }, + "node_modules/xss/node_modules/commander": { + "version": "2.20.3", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", + "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + }, + "node_modules/xstate": { + "version": "4.32.0", + "resolved": "https://registry.npmjs.org/xstate/-/xstate-4.32.0.tgz", + "integrity": "sha512-62gETqwnw4pBRe+tVWMt8hLgWEU8lq2qO8VN5PWmTELceRVt3I1bu1cwdraVRHUn4Bb2lnhNzn1A73oShuC+8g==" + }, + "node_modules/xtend": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/xtend/-/xtend-4.0.2.tgz", + "integrity": "sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ==", + "engines": { + "node": ">=0.4" + } + }, + "node_modules/xxhash-wasm": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/xxhash-wasm/-/xxhash-wasm-0.4.2.tgz", + "integrity": "sha512-/eyHVRJQCirEkSZ1agRSCwriMhwlyUcFkXD5TPVSLP+IPzjsqMVzZwdoczLp1SoQU0R3dxz1RpIK+4YNQbCVOA==" + }, + "node_modules/y18n": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-4.0.3.tgz", + "integrity": "sha512-JKhqTOwSrqNA1NY5lSztJ1GrBiUodLMmIZuLiDaMRJ+itFd+ABVE8XBjOvIWL+rSqNDC74LCSFmlb/U4UZ4hJQ==" + }, + "node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + }, + "node_modules/yaml": { + "version": "1.10.2", + "resolved": "https://registry.npmjs.org/yaml/-/yaml-1.10.2.tgz", + "integrity": "sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg==", + "engines": { + "node": ">= 6" + } + }, + "node_modules/yaml-loader": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/yaml-loader/-/yaml-loader-0.6.0.tgz", + "integrity": "sha512-1bNiLelumURyj+zvVHOv8Y3dpCri0F2S+DCcmps0pA1zWRLjS+FhZQg4o3aUUDYESh73+pKZNI18bj7stpReow==", + "dependencies": { + "loader-utils": "^1.4.0", + "yaml": "^1.8.3" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/yaml-loader/node_modules/json5": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "dependencies": { + "minimist": "^1.2.0" + }, + "bin": { + "json5": "lib/cli.js" + } + }, + "node_modules/yaml-loader/node_modules/loader-utils": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-1.4.0.tgz", + "integrity": "sha512-qH0WSMBtn/oHuwjy/NucEgbx5dbxxnxup9s4PVXJUDHZBQY+s0NWA9rJf53RBnQZxfch7euUui7hpoAPvALZdA==", + "dependencies": { + "big.js": "^5.2.2", + "emojis-list": "^3.0.0", + "json5": "^1.0.1" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/yargs": { + "version": "15.4.1", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-15.4.1.tgz", + "integrity": "sha512-aePbxDmcYW++PaqBsJ+HYUFwCdv4LVvdnhBy78E57PIor8/OVvhMrADFFEDh8DHDFRv/O9i3lPhsENjO7QX0+A==", + "dependencies": { + "cliui": "^6.0.0", + "decamelize": "^1.2.0", + "find-up": "^4.1.0", + "get-caller-file": "^2.0.1", + "require-directory": "^2.1.1", + "require-main-filename": "^2.0.0", + "set-blocking": "^2.0.0", + "string-width": "^4.2.0", + "which-module": "^2.0.0", + "y18n": "^4.0.0", + "yargs-parser": "^18.1.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/yargs-parser": { + "version": "18.1.3", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-18.1.3.tgz", + "integrity": "sha512-o50j0JeToy/4K6OZcaQmW6lyXXKhq7csREXcDwk2omFPJEwUNOVtJKvmDr9EI1fAJZUyZcRF7kxGBWmRXudrCQ==", + "dependencies": { + "camelcase": "^5.0.0", + "decamelize": "^1.2.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/yeast": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/yeast/-/yeast-0.1.2.tgz", + "integrity": "sha1-AI4G2AlDIMNy28L47XagymyKxBk=" + }, + "node_modules/yn": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/yn/-/yn-3.1.1.tgz", + "integrity": "sha512-Ux4ygGWsu2c7isFWe8Yu1YluJmqVhxqK2cLXNQA5AcC3QfbGNpM7fu0Y8b/z16pXLnFxZYvWhd3fhBY9DLmC6Q==", + "engines": { + "node": ">=6" + } + }, + "node_modules/yocto-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", + "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", + "engines": { + "node": ">=10" + } + }, + "node_modules/yoga-layout-prebuilt": { + "version": "1.10.0", + "resolved": "https://registry.npmjs.org/yoga-layout-prebuilt/-/yoga-layout-prebuilt-1.10.0.tgz", + "integrity": "sha512-YnOmtSbv4MTf7RGJMK0FvZ+KD8OEe/J5BNnR0GHhD8J/XcG/Qvxgszm0Un6FTHWW4uHlTgP0IztiXQnGyIR45g==", + "dependencies": { + "@types/yoga-layout": "1.9.2" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/yurnalist": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/yurnalist/-/yurnalist-2.1.0.tgz", + "integrity": "sha512-PgrBqosQLM3gN2xBFIMDLACRTV9c365VqityKKpSTWpwR+U4LAFR3rSVyEoscWlu3EzX9+Y0I86GXUKxpHFl6w==", + "dependencies": { + "chalk": "^2.4.2", + "inquirer": "^7.0.0", + "is-ci": "^2.0.0", + "read": "^1.0.7", + "strip-ansi": "^5.2.0" + }, + "engines": { + "node": ">=4.0.0" + } + }, + "node_modules/yurnalist/node_modules/ansi-regex": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-4.1.1.tgz", + "integrity": "sha512-ILlv4k/3f6vfQ4OoP2AGvirOktlQ98ZEL1k9FaQjxa3L1abBgbuTDAdPOpvbGncC0BTVQrl+OM8xZGK6tWXt7g==", + "engines": { + "node": ">=6" + } + }, + "node_modules/yurnalist/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/yurnalist/node_modules/strip-ansi": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-5.2.0.tgz", + "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", + "dependencies": { + "ansi-regex": "^4.1.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/zwitch": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/zwitch/-/zwitch-1.0.5.tgz", + "integrity": "sha512-V50KMwwzqJV0NpZIZFwfOD5/lyny3WlSzRiXgA0G7VUnRlqttta1L6UQIHzd6EuBY/cHGfwTIck7w1yH6Q5zUw==" + } + }, + "dependencies": { + "@ampproject/remapping": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.2.0.tgz", + "integrity": "sha512-qRmjj8nj9qmLTQXXmaR1cck3UXSRMPrbsLJAasZpF+t3riI71BXed5ebIOYwQntykeZuhjsdweEc9BxH5Jc26w==", + "requires": { + "@jridgewell/gen-mapping": "^0.1.0", + "@jridgewell/trace-mapping": "^0.3.9" + } + }, + "@ardatan/aggregate-error": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/@ardatan/aggregate-error/-/aggregate-error-0.0.6.tgz", + "integrity": "sha512-vyrkEHG1jrukmzTPtyWB4NLPauUw5bQeg4uhn8f+1SSynmrOcyvlb1GKQjjgoBzElLdfXCRYX8UnBlhklOHYRQ==", + "requires": { + "tslib": "~2.0.1" + }, + "dependencies": { + "tslib": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.0.3.tgz", + "integrity": "sha512-uZtkfKblCEQtZKBF6EBXVZeQNl82yqtDQdv+eck8u7tdPxjLu2/lp5/uPW+um2tpuxINHWy3GhiccY7QgEaVHQ==" + } + } + }, + "@babel/code-frame": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.16.7.tgz", + "integrity": "sha512-iAXqUn8IIeBTNd72xsFlgaXHkMBMt6y4HJp1tIaK465CWLT/fG1aqB7ykr95gHHmlBdGbFeWWfyB4NJJ0nmeIg==", + "requires": { + "@babel/highlight": "^7.16.7" + } + }, + "@babel/compat-data": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.17.10.tgz", + "integrity": "sha512-GZt/TCsG70Ms19gfZO1tM4CVnXsPgEPBCpJu+Qz3L0LUDsY5nZqFZglIoPC1kIYOtNBZlrnFT+klg12vFGZXrw==" + }, + "@babel/core": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.17.10.tgz", + "integrity": "sha512-liKoppandF3ZcBnIYFjfSDHZLKdLHGJRkoWtG8zQyGJBQfIYobpnVGI5+pLBNtS6psFLDzyq8+h5HiVljW9PNA==", + "requires": { + "@ampproject/remapping": "^2.1.0", + "@babel/code-frame": "^7.16.7", + "@babel/generator": "^7.17.10", + "@babel/helper-compilation-targets": "^7.17.10", + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helpers": "^7.17.9", + "@babel/parser": "^7.17.10", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.10", + "@babel/types": "^7.17.10", + "convert-source-map": "^1.7.0", + "debug": "^4.1.0", + "gensync": "^1.0.0-beta.2", + "json5": "^2.2.1", + "semver": "^6.3.0" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + }, + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/eslint-parser": { + "version": "7.17.0", + "resolved": "https://registry.npmjs.org/@babel/eslint-parser/-/eslint-parser-7.17.0.tgz", + "integrity": "sha512-PUEJ7ZBXbRkbq3qqM/jZ2nIuakUBqCYc7Qf52Lj7dlZ6zERnqisdHioL0l4wwQZnmskMeasqUNzLBFKs3nylXA==", + "requires": { + "eslint-scope": "^5.1.1", + "eslint-visitor-keys": "^2.1.0", + "semver": "^6.3.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/generator": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.17.10.tgz", + "integrity": "sha512-46MJZZo9y3o4kmhBVc7zW7i8dtR1oIK/sdO5NcfcZRhTGYi+KKJRtHNgsU6c4VUcJmUNV/LQdebD/9Dlv4K+Tg==", + "requires": { + "@babel/types": "^7.17.10", + "@jridgewell/gen-mapping": "^0.1.0", + "jsesc": "^2.5.1" + } + }, + "@babel/helper-annotate-as-pure": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.16.7.tgz", + "integrity": "sha512-s6t2w/IPQVTAET1HitoowRGXooX8mCgtuP5195wD/QJPV6wYjpujCGF7JuMODVX2ZAJOf1GT6DT9MHEZvLOFSw==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-builder-binary-assignment-operator-visitor": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.16.7.tgz", + "integrity": "sha512-C6FdbRaxYjwVu/geKW4ZeQ0Q31AftgRcdSnZ5/jsH6BzCJbtvXvhpfkbkThYSuutZA7nCXpPR6AD9zd1dprMkA==", + "requires": { + "@babel/helper-explode-assignable-expression": "^7.16.7", + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-compilation-targets": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.17.10.tgz", + "integrity": "sha512-gh3RxjWbauw/dFiU/7whjd0qN9K6nPJMqe6+Er7rOavFh0CQUSwhAE3IcTho2rywPJFxej6TUUHDkWcYI6gGqQ==", + "requires": { + "@babel/compat-data": "^7.17.10", + "@babel/helper-validator-option": "^7.16.7", + "browserslist": "^4.20.2", + "semver": "^6.3.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/helper-create-class-features-plugin": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.17.9.tgz", + "integrity": "sha512-kUjip3gruz6AJKOq5i3nC6CoCEEF/oHH3cp6tOZhB+IyyyPyW0g1Gfsxn3mkk6S08pIA2y8GQh609v9G/5sHVQ==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.17.9", + "@babel/helper-member-expression-to-functions": "^7.17.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7" + } + }, + "@babel/helper-create-regexp-features-plugin": { + "version": "7.17.0", + "resolved": "https://registry.npmjs.org/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.17.0.tgz", + "integrity": "sha512-awO2So99wG6KnlE+TPs6rn83gCz5WlEePJDTnLEqbchMVrBeAujURVphRdigsk094VhvZehFoNOihSlcBjwsXA==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "regexpu-core": "^5.0.1" + } + }, + "@babel/helper-define-polyfill-provider": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.3.1.tgz", + "integrity": "sha512-J9hGMpJQmtWmj46B3kBHmL38UhJGhYX7eqkcq+2gsstyYt341HmPeWspihX43yVRA0mS+8GGk2Gckc7bY/HCmA==", + "requires": { + "@babel/helper-compilation-targets": "^7.13.0", + "@babel/helper-module-imports": "^7.12.13", + "@babel/helper-plugin-utils": "^7.13.0", + "@babel/traverse": "^7.13.0", + "debug": "^4.1.1", + "lodash.debounce": "^4.0.8", + "resolve": "^1.14.2", + "semver": "^6.1.2" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + }, + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/helper-environment-visitor": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.16.7.tgz", + "integrity": "sha512-SLLb0AAn6PkUeAfKJCCOl9e1R53pQlGAfc4y4XuMRZfqeMYLE0dM1LMhqbGAlGQY0lfw5/ohoYWAe9V1yibRag==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-explode-assignable-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-explode-assignable-expression/-/helper-explode-assignable-expression-7.16.7.tgz", + "integrity": "sha512-KyUenhWMC8VrxzkGP0Jizjo4/Zx+1nNZhgocs+gLzyZyB8SHidhoq9KK/8Ato4anhwsivfkBLftky7gvzbZMtQ==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-function-name": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.17.9.tgz", + "integrity": "sha512-7cRisGlVtiVqZ0MW0/yFB4atgpGLWEHUVYnb448hZK4x+vih0YO5UoS11XIYtZYqHd0dIPMdUSv8q5K4LdMnIg==", + "requires": { + "@babel/template": "^7.16.7", + "@babel/types": "^7.17.0" + } + }, + "@babel/helper-hoist-variables": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.16.7.tgz", + "integrity": "sha512-m04d/0Op34H5v7pbZw6pSKP7weA6lsMvfiIAMeIvkY/R4xQtBSMFEigu9QTZ2qB/9l22vsxtM8a+Q8CzD255fg==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-member-expression-to-functions": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.17.7.tgz", + "integrity": "sha512-thxXgnQ8qQ11W2wVUObIqDL4p148VMxkt5T/qpN5k2fboRyzFGFmKsTGViquyM5QHKUy48OZoca8kw4ajaDPyw==", + "requires": { + "@babel/types": "^7.17.0" + } + }, + "@babel/helper-module-imports": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.16.7.tgz", + "integrity": "sha512-LVtS6TqjJHFc+nYeITRo6VLXve70xmq7wPhWTqDJusJEgGmkAACWwMiTNrvfoQo6hEhFwAIixNkvB0jPXDL8Wg==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-module-transforms": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.17.7.tgz", + "integrity": "sha512-VmZD99F3gNTYB7fJRDTi+u6l/zxY0BE6OIxPSU7a50s6ZUQkHwSDmV92FfM+oCG0pZRVojGYhkR8I0OGeCVREw==", + "requires": { + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-simple-access": "^7.17.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "@babel/helper-validator-identifier": "^7.16.7", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.3", + "@babel/types": "^7.17.0" + } + }, + "@babel/helper-optimise-call-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.16.7.tgz", + "integrity": "sha512-EtgBhg7rd/JcnpZFXpBy0ze1YRfdm7BnBX4uKMBd3ixa3RGAE002JZB66FJyNH7g0F38U05pXmA5P8cBh7z+1w==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-plugin-utils": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.16.7.tgz", + "integrity": "sha512-Qg3Nk7ZxpgMrsox6HreY1ZNKdBq7K72tDSliA6dCl5f007jR4ne8iD5UzuNnCJH2xBf2BEEVGr+/OL6Gdp7RxA==" + }, + "@babel/helper-remap-async-to-generator": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.16.8.tgz", + "integrity": "sha512-fm0gH7Flb8H51LqJHy3HJ3wnE1+qtYR2A99K06ahwrawLdOFsCEWjZOrYricXJHoPSudNKxrMBUPEIPxiIIvBw==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-wrap-function": "^7.16.8", + "@babel/types": "^7.16.8" + } + }, + "@babel/helper-replace-supers": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-replace-supers/-/helper-replace-supers-7.16.7.tgz", + "integrity": "sha512-y9vsWilTNaVnVh6xiJfABzsNpgDPKev9HnAgz6Gb1p6UUwf9NepdlsV7VXGCftJM+jqD5f7JIEubcpLjZj5dBw==", + "requires": { + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-member-expression-to-functions": "^7.16.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/traverse": "^7.16.7", + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-simple-access": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.17.7.tgz", + "integrity": "sha512-txyMCGroZ96i+Pxr3Je3lzEJjqwaRC9buMUgtomcrLe5Nd0+fk1h0LLA+ixUF5OW7AhHuQ7Es1WcQJZmZsz2XA==", + "requires": { + "@babel/types": "^7.17.0" + } + }, + "@babel/helper-skip-transparent-expression-wrappers": { + "version": "7.16.0", + "resolved": "https://registry.npmjs.org/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.16.0.tgz", + "integrity": "sha512-+il1gTy0oHwUsBQZyJvukbB4vPMdcYBrFHa0Uc4AizLxbq6BOYC51Rv4tWocX9BLBDLZ4kc6qUFpQ6HRgL+3zw==", + "requires": { + "@babel/types": "^7.16.0" + } + }, + "@babel/helper-split-export-declaration": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.7.tgz", + "integrity": "sha512-xbWoy/PFoxSWazIToT9Sif+jJTlrMcndIsaOKvTA6u7QEo7ilkRZpjew18/W3c7nm8fXdUDXh02VXTbZ0pGDNw==", + "requires": { + "@babel/types": "^7.16.7" + } + }, + "@babel/helper-validator-identifier": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.16.7.tgz", + "integrity": "sha512-hsEnFemeiW4D08A5gUAZxLBTXpZ39P+a+DGDsHw1yxqyQ/jzFEnxf5uTEGp+3bzAbNOxU1paTgYS4ECU/IgfDw==" + }, + "@babel/helper-validator-option": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.16.7.tgz", + "integrity": "sha512-TRtenOuRUVo9oIQGPC5G9DgK4743cdxvtOw0weQNpZXaS16SCBi5MNjZF8vba3ETURjZpTbVn7Vvcf2eAwFozQ==" + }, + "@babel/helper-wrap-function": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/helper-wrap-function/-/helper-wrap-function-7.16.8.tgz", + "integrity": "sha512-8RpyRVIAW1RcDDGTA+GpPAwV22wXCfKOoM9bet6TLkGIFTkRQSkH1nMQ5Yet4MpoXe1ZwHPVtNasc2w0uZMqnw==", + "requires": { + "@babel/helper-function-name": "^7.16.7", + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.16.8", + "@babel/types": "^7.16.8" + } + }, + "@babel/helpers": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.17.9.tgz", + "integrity": "sha512-cPCt915ShDWUEzEp3+UNRktO2n6v49l5RSnG9M5pS24hA+2FAc5si+Pn1i4VVbQQ+jh+bIZhPFQOJOzbrOYY1Q==", + "requires": { + "@babel/template": "^7.16.7", + "@babel/traverse": "^7.17.9", + "@babel/types": "^7.17.0" + } + }, + "@babel/highlight": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.17.9.tgz", + "integrity": "sha512-J9PfEKCbFIv2X5bjTMiZu6Vf341N05QIY+d6FvVKynkG1S7G0j3I0QoRtWIrXhZ+/Nlb5Q0MzqL7TokEJ5BNHg==", + "requires": { + "@babel/helper-validator-identifier": "^7.16.7", + "chalk": "^2.0.0", + "js-tokens": "^4.0.0" + }, + "dependencies": { + "chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + } + } + }, + "@babel/parser": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.17.10.tgz", + "integrity": "sha512-n2Q6i+fnJqzOaq2VkdXxy2TCPCWQZHiCo0XqmrCvDWcZQKRyZzYi4Z0yxlBuN0w+r2ZHmre+Q087DSrw3pbJDQ==" + }, + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.16.7.tgz", + "integrity": "sha512-anv/DObl7waiGEnC24O9zqL0pSuI9hljihqiDuFHC8d7/bjr/4RLGPWuc8rYOff/QPzbEPSkzG8wGG9aDuhHRg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.16.7.tgz", + "integrity": "sha512-di8vUHRdf+4aJ7ltXhaDbPoszdkh59AQtJM5soLsuHpQJdFQZOA4uGj0V2u/CZ8bJ/u8ULDL5yq6FO/bCXnKHw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0", + "@babel/plugin-proposal-optional-chaining": "^7.16.7" + } + }, + "@babel/plugin-proposal-async-generator-functions": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-async-generator-functions/-/plugin-proposal-async-generator-functions-7.16.8.tgz", + "integrity": "sha512-71YHIvMuiuqWJQkebWJtdhQTfd4Q4mF76q2IX37uZPkG9+olBxsX+rH1vkhFto4UeJZ9dPY2s+mDvhDm1u2BGQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-remap-async-to-generator": "^7.16.8", + "@babel/plugin-syntax-async-generators": "^7.8.4" + } + }, + "@babel/plugin-proposal-class-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-class-properties/-/plugin-proposal-class-properties-7.16.7.tgz", + "integrity": "sha512-IobU0Xme31ewjYOShSIqd/ZGM/r/cuOz2z0MDbNrhF5FW+ZVgi0f2lyeoj9KFPDOAqsYxmLWZte1WOwlvY9aww==", + "requires": { + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-proposal-class-static-block": { + "version": "7.17.6", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-class-static-block/-/plugin-proposal-class-static-block-7.17.6.tgz", + "integrity": "sha512-X/tididvL2zbs7jZCeeRJ8167U/+Ac135AM6jCAx6gYXDUviZV5Ku9UDvWS2NCuWlFjIRXklYhwo6HhAC7ETnA==", + "requires": { + "@babel/helper-create-class-features-plugin": "^7.17.6", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-class-static-block": "^7.14.5" + } + }, + "@babel/plugin-proposal-dynamic-import": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-dynamic-import/-/plugin-proposal-dynamic-import-7.16.7.tgz", + "integrity": "sha512-I8SW9Ho3/8DRSdmDdH3gORdyUuYnk1m4cMxUAdu5oy4n3OfN8flDEH+d60iG7dUfi0KkYwSvoalHzzdRzpWHTg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-dynamic-import": "^7.8.3" + } + }, + "@babel/plugin-proposal-export-namespace-from": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-export-namespace-from/-/plugin-proposal-export-namespace-from-7.16.7.tgz", + "integrity": "sha512-ZxdtqDXLRGBL64ocZcs7ovt71L3jhC1RGSyR996svrCi3PYqHNkb3SwPJCs8RIzD86s+WPpt2S73+EHCGO+NUA==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-export-namespace-from": "^7.8.3" + } + }, + "@babel/plugin-proposal-json-strings": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-json-strings/-/plugin-proposal-json-strings-7.16.7.tgz", + "integrity": "sha512-lNZ3EEggsGY78JavgbHsK9u5P3pQaW7k4axlgFLYkMd7UBsiNahCITShLjNQschPyjtO6dADrL24757IdhBrsQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-json-strings": "^7.8.3" + } + }, + "@babel/plugin-proposal-logical-assignment-operators": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-logical-assignment-operators/-/plugin-proposal-logical-assignment-operators-7.16.7.tgz", + "integrity": "sha512-K3XzyZJGQCr00+EtYtrDjmwX7o7PLK6U9bi1nCwkQioRFVUv6dJoxbQjtWVtP+bCPy82bONBKG8NPyQ4+i6yjg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-logical-assignment-operators": "^7.10.4" + } + }, + "@babel/plugin-proposal-nullish-coalescing-operator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-nullish-coalescing-operator/-/plugin-proposal-nullish-coalescing-operator-7.16.7.tgz", + "integrity": "sha512-aUOrYU3EVtjf62jQrCj63pYZ7k6vns2h/DQvHPWGmsJRYzWXZ6/AsfgpiRy6XiuIDADhJzP2Q9MwSMKauBQ+UQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3" + } + }, + "@babel/plugin-proposal-numeric-separator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-numeric-separator/-/plugin-proposal-numeric-separator-7.16.7.tgz", + "integrity": "sha512-vQgPMknOIgiuVqbokToyXbkY/OmmjAzr/0lhSIbG/KmnzXPGwW/AdhdKpi+O4X/VkWiWjnkKOBiqJrTaC98VKw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-numeric-separator": "^7.10.4" + } + }, + "@babel/plugin-proposal-object-rest-spread": { + "version": "7.17.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-object-rest-spread/-/plugin-proposal-object-rest-spread-7.17.3.tgz", + "integrity": "sha512-yuL5iQA/TbZn+RGAfxQXfi7CNLmKi1f8zInn4IgobuCWcAb7i+zj4TYzQ9l8cEzVyJ89PDGuqxK1xZpUDISesw==", + "requires": { + "@babel/compat-data": "^7.17.0", + "@babel/helper-compilation-targets": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-object-rest-spread": "^7.8.3", + "@babel/plugin-transform-parameters": "^7.16.7" + } + }, + "@babel/plugin-proposal-optional-catch-binding": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-optional-catch-binding/-/plugin-proposal-optional-catch-binding-7.16.7.tgz", + "integrity": "sha512-eMOH/L4OvWSZAE1VkHbr1vckLG1WUcHGJSLqqQwl2GaUqG6QjddvrOaTUMNYiv77H5IKPMZ9U9P7EaHwvAShfA==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-optional-catch-binding": "^7.8.3" + } + }, + "@babel/plugin-proposal-optional-chaining": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-optional-chaining/-/plugin-proposal-optional-chaining-7.16.7.tgz", + "integrity": "sha512-eC3xy+ZrUcBtP7x+sq62Q/HYd674pPTb/77XZMb5wbDPGWIdUbSr4Agr052+zaUPSb+gGRnjxXfKFvx5iMJ+DA==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0", + "@babel/plugin-syntax-optional-chaining": "^7.8.3" + } + }, + "@babel/plugin-proposal-private-methods": { + "version": "7.16.11", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-private-methods/-/plugin-proposal-private-methods-7.16.11.tgz", + "integrity": "sha512-F/2uAkPlXDr8+BHpZvo19w3hLFKge+k75XUprE6jaqKxjGkSYcK+4c+bup5PdW/7W/Rpjwql7FTVEDW+fRAQsw==", + "requires": { + "@babel/helper-create-class-features-plugin": "^7.16.10", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-proposal-private-property-in-object": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.16.7.tgz", + "integrity": "sha512-rMQkjcOFbm+ufe3bTZLyOfsOUOxyvLXZJCTARhJr+8UMSoZmqTe1K1BgkFcrW37rAchWg57yI69ORxiWvUINuQ==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-private-property-in-object": "^7.14.5" + } + }, + "@babel/plugin-proposal-unicode-property-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-proposal-unicode-property-regex/-/plugin-proposal-unicode-property-regex-7.16.7.tgz", + "integrity": "sha512-QRK0YI/40VLhNVGIjRNAAQkEHws0cswSdFFjpFyt943YmJIU1da9uW63Iu6NFV6CxTZW5eTDCrwZUstBWgp/Rg==", + "requires": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-syntax-async-generators": { + "version": "7.8.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz", + "integrity": "sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-class-properties": { + "version": "7.12.13", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz", + "integrity": "sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA==", + "requires": { + "@babel/helper-plugin-utils": "^7.12.13" + } + }, + "@babel/plugin-syntax-class-static-block": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz", + "integrity": "sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw==", + "requires": { + "@babel/helper-plugin-utils": "^7.14.5" + } + }, + "@babel/plugin-syntax-dynamic-import": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz", + "integrity": "sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-export-namespace-from": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz", + "integrity": "sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.3" + } + }, + "@babel/plugin-syntax-flow": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-flow/-/plugin-syntax-flow-7.16.7.tgz", + "integrity": "sha512-UDo3YGQO0jH6ytzVwgSLv9i/CzMcUjbKenL67dTrAZPPv6GFAtDhe6jqnvmoKzC/7htNTohhos+onPtDMqJwaQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-syntax-json-strings": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz", + "integrity": "sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-jsx": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.16.7.tgz", + "integrity": "sha512-Esxmk7YjA8QysKeT3VhTXvF6y77f/a91SIs4pWb4H2eWGQkCKFgQaG6hdoEVZtGsrAcb2K5BW66XsOErD4WU3Q==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-syntax-logical-assignment-operators": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz", + "integrity": "sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig==", + "requires": { + "@babel/helper-plugin-utils": "^7.10.4" + } + }, + "@babel/plugin-syntax-nullish-coalescing-operator": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz", + "integrity": "sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-numeric-separator": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz", + "integrity": "sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug==", + "requires": { + "@babel/helper-plugin-utils": "^7.10.4" + } + }, + "@babel/plugin-syntax-object-rest-spread": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz", + "integrity": "sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-optional-catch-binding": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz", + "integrity": "sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-optional-chaining": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz", + "integrity": "sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg==", + "requires": { + "@babel/helper-plugin-utils": "^7.8.0" + } + }, + "@babel/plugin-syntax-private-property-in-object": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz", + "integrity": "sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg==", + "requires": { + "@babel/helper-plugin-utils": "^7.14.5" + } + }, + "@babel/plugin-syntax-top-level-await": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz", + "integrity": "sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw==", + "requires": { + "@babel/helper-plugin-utils": "^7.14.5" + } + }, + "@babel/plugin-syntax-typescript": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.17.10.tgz", + "integrity": "sha512-xJefea1DWXW09pW4Tm9bjwVlPDyYA2it3fWlmEjpYz6alPvTUjL0EOzNzI/FEOyI3r4/J7uVH5UqKgl1TQ5hqQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-arrow-functions": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.16.7.tgz", + "integrity": "sha512-9ffkFFMbvzTvv+7dTp/66xvZAWASuPD5Tl9LK3Z9vhOmANo6j94rik+5YMBt4CwHVMWLWpMsriIc2zsa3WW3xQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-async-to-generator": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.16.8.tgz", + "integrity": "sha512-MtmUmTJQHCnyJVrScNzNlofQJ3dLFuobYn3mwOTKHnSCMtbNsqvF71GQmJfFjdrXSsAA7iysFmYWw4bXZ20hOg==", + "requires": { + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-remap-async-to-generator": "^7.16.8" + } + }, + "@babel/plugin-transform-block-scoped-functions": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.16.7.tgz", + "integrity": "sha512-JUuzlzmF40Z9cXyytcbZEZKckgrQzChbQJw/5PuEHYeqzCsvebDx0K0jWnIIVcmmDOAVctCgnYs0pMcrYj2zJg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-block-scoping": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.16.7.tgz", + "integrity": "sha512-ObZev2nxVAYA4bhyusELdo9hb3H+A56bxH3FZMbEImZFiEDYVHXQSJ1hQKFlDnlt8G9bBrCZ5ZpURZUrV4G5qQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-classes": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-classes/-/plugin-transform-classes-7.16.7.tgz", + "integrity": "sha512-WY7og38SFAGYRe64BrjKf8OrE6ulEHtr5jEYaZMwox9KebgqPi67Zqz8K53EKk1fFEJgm96r32rkKZ3qA2nCWQ==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.16.7", + "@babel/helper-optimise-call-expression": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "globals": "^11.1.0" + } + }, + "@babel/plugin-transform-computed-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.16.7.tgz", + "integrity": "sha512-gN72G9bcmenVILj//sv1zLNaPyYcOzUho2lIJBMh/iakJ9ygCo/hEF9cpGb61SCMEDxbbyBoVQxrt+bWKu5KGw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-destructuring": { + "version": "7.17.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.17.7.tgz", + "integrity": "sha512-XVh0r5yq9sLR4vZ6eVZe8FKfIcSgaTBxVBRSYokRj2qksf6QerYnTxz9/GTuKTH/n/HwLP7t6gtlybHetJ/6hQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-dotall-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.16.7.tgz", + "integrity": "sha512-Lyttaao2SjZF6Pf4vk1dVKv8YypMpomAbygW+mU5cYP3S5cWTfCJjG8xV6CFdzGFlfWK81IjL9viiTvpb6G7gQ==", + "requires": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-duplicate-keys": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.16.7.tgz", + "integrity": "sha512-03DvpbRfvWIXyK0/6QiR1KMTWeT6OcQ7tbhjrXyFS02kjuX/mu5Bvnh5SDSWHxyawit2g5aWhKwI86EE7GUnTw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-exponentiation-operator": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.16.7.tgz", + "integrity": "sha512-8UYLSlyLgRixQvlYH3J2ekXFHDFLQutdy7FfFAMm3CPZ6q9wHCwnUyiXpQCe3gVVnQlHc5nsuiEVziteRNTXEA==", + "requires": { + "@babel/helper-builder-binary-assignment-operator-visitor": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-flow-strip-types": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-flow-strip-types/-/plugin-transform-flow-strip-types-7.16.7.tgz", + "integrity": "sha512-mzmCq3cNsDpZZu9FADYYyfZJIOrSONmHcop2XEKPdBNMa4PDC4eEvcOvzZaCNcjKu72v0XQlA5y1g58aLRXdYg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-flow": "^7.16.7" + } + }, + "@babel/plugin-transform-for-of": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.16.7.tgz", + "integrity": "sha512-/QZm9W92Ptpw7sjI9Nx1mbcsWz33+l8kuMIQnDwgQBG5s3fAfQvkRjQ7NqXhtNcKOnPkdICmUHyCaWW06HCsqg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-function-name": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.16.7.tgz", + "integrity": "sha512-SU/C68YVwTRxqWj5kgsbKINakGag0KTgq9f2iZEXdStoAbOzLHEBRYzImmA6yFo8YZhJVflvXmIHUO7GWHmxxA==", + "requires": { + "@babel/helper-compilation-targets": "^7.16.7", + "@babel/helper-function-name": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-literals/-/plugin-transform-literals-7.16.7.tgz", + "integrity": "sha512-6tH8RTpTWI0s2sV6uq3e/C9wPo4PTqqZps4uF0kzQ9/xPLFQtipynvmT1g/dOfEJ+0EQsHhkQ/zyRId8J2b8zQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-member-expression-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.16.7.tgz", + "integrity": "sha512-mBruRMbktKQwbxaJof32LT9KLy2f3gH+27a5XSuXo6h7R3vqltl0PgZ80C8ZMKw98Bf8bqt6BEVi3svOh2PzMw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-modules-amd": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.16.7.tgz", + "integrity": "sha512-KaaEtgBL7FKYwjJ/teH63oAmE3lP34N3kshz8mm4VMAw7U3PxjVwwUmxEFksbgsNUaO3wId9R2AVQYSEGRa2+g==", + "requires": { + "@babel/helper-module-transforms": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + } + }, + "@babel/plugin-transform-modules-commonjs": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.17.9.tgz", + "integrity": "sha512-2TBFd/r2I6VlYn0YRTz2JdazS+FoUuQ2rIFHoAxtyP/0G3D82SBLaRq9rnUkpqlLg03Byfl/+M32mpxjO6KaPw==", + "requires": { + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-simple-access": "^7.17.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + } + }, + "@babel/plugin-transform-modules-systemjs": { + "version": "7.17.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.17.8.tgz", + "integrity": "sha512-39reIkMTUVagzgA5x88zDYXPCMT6lcaRKs1+S9K6NKBPErbgO/w/kP8GlNQTC87b412ZTlmNgr3k2JrWgHH+Bw==", + "requires": { + "@babel/helper-hoist-variables": "^7.16.7", + "@babel/helper-module-transforms": "^7.17.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-identifier": "^7.16.7", + "babel-plugin-dynamic-import-node": "^2.3.3" + } + }, + "@babel/plugin-transform-modules-umd": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.16.7.tgz", + "integrity": "sha512-EMh7uolsC8O4xhudF2F6wedbSHm1HHZ0C6aJ7K67zcDNidMzVcxWdGr+htW9n21klm+bOn+Rx4CBsAntZd3rEQ==", + "requires": { + "@babel/helper-module-transforms": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-named-capturing-groups-regex": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.17.10.tgz", + "integrity": "sha512-v54O6yLaJySCs6mGzaVOUw9T967GnH38T6CQSAtnzdNPwu84l2qAjssKzo/WSO8Yi7NF+7ekm5cVbF/5qiIgNA==", + "requires": { + "@babel/helper-create-regexp-features-plugin": "^7.17.0" + } + }, + "@babel/plugin-transform-new-target": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.16.7.tgz", + "integrity": "sha512-xiLDzWNMfKoGOpc6t3U+etCE2yRnn3SM09BXqWPIZOBpL2gvVrBWUKnsJx0K/ADi5F5YC5f8APFfWrz25TdlGg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-object-super": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.16.7.tgz", + "integrity": "sha512-14J1feiQVWaGvRxj2WjyMuXS2jsBkgB3MdSN5HuC2G5nRspa5RK9COcs82Pwy5BuGcjb+fYaUj94mYcOj7rCvw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-replace-supers": "^7.16.7" + } + }, + "@babel/plugin-transform-parameters": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.16.7.tgz", + "integrity": "sha512-AT3MufQ7zZEhU2hwOA11axBnExW0Lszu4RL/tAlUJBuNoRak+wehQW8h6KcXOcgjY42fHtDxswuMhMjFEuv/aw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-property-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.16.7.tgz", + "integrity": "sha512-z4FGr9NMGdoIl1RqavCqGG+ZuYjfZ/hkCIeuH6Do7tXmSm0ls11nYVSJqFEUOSJbDab5wC6lRE/w6YjVcr6Hqw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-react-display-name": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.16.7.tgz", + "integrity": "sha512-qgIg8BcZgd0G/Cz916D5+9kqX0c7nPZyXaP8R2tLNN5tkyIZdG5fEwBrxwplzSnjC1jvQmyMNVwUCZPcbGY7Pg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-react-jsx": { + "version": "7.17.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.17.3.tgz", + "integrity": "sha512-9tjBm4O07f7mzKSIlEmPdiE6ub7kfIe6Cd+w+oQebpATfTQMAgW+YOuWxogbKVTulA+MEO7byMeIUtQ1z+z+ZQ==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-jsx": "^7.16.7", + "@babel/types": "^7.17.0" + } + }, + "@babel/plugin-transform-react-jsx-development": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.16.7.tgz", + "integrity": "sha512-RMvQWvpla+xy6MlBpPlrKZCMRs2AGiHOGHY3xRwl0pEeim348dDyxeH4xBsMPbIMhujeq7ihE702eM2Ew0Wo+A==", + "requires": { + "@babel/plugin-transform-react-jsx": "^7.16.7" + } + }, + "@babel/plugin-transform-react-pure-annotations": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.16.7.tgz", + "integrity": "sha512-hs71ToC97k3QWxswh2ElzMFABXHvGiJ01IB1TbYQDGeWRKWz/MPUTh5jGExdHvosYKpnJW5Pm3S4+TA3FyX+GA==", + "requires": { + "@babel/helper-annotate-as-pure": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-regenerator": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.17.9.tgz", + "integrity": "sha512-Lc2TfbxR1HOyn/c6b4Y/b6NHoTb67n/IoWLxTu4kC7h4KQnWlhCq2S8Tx0t2SVvv5Uu87Hs+6JEJ5kt2tYGylQ==", + "requires": { + "regenerator-transform": "^0.15.0" + } + }, + "@babel/plugin-transform-reserved-words": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.16.7.tgz", + "integrity": "sha512-KQzzDnZ9hWQBjwi5lpY5v9shmm6IVG0U9pB18zvMu2i4H90xpT4gmqwPYsn8rObiadYe2M0gmgsiOIF5A/2rtg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-runtime": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.17.10.tgz", + "integrity": "sha512-6jrMilUAJhktTr56kACL8LnWC5hx3Lf27BS0R0DSyW/OoJfb/iTHeE96V3b1dgKG3FSFdd/0culnYWMkjcKCig==", + "requires": { + "@babel/helper-module-imports": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "babel-plugin-polyfill-corejs2": "^0.3.0", + "babel-plugin-polyfill-corejs3": "^0.5.0", + "babel-plugin-polyfill-regenerator": "^0.3.0", + "semver": "^6.3.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/plugin-transform-shorthand-properties": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.16.7.tgz", + "integrity": "sha512-hah2+FEnoRoATdIb05IOXf+4GzXYTq75TVhIn1PewihbpyrNWUt2JbudKQOETWw6QpLe+AIUpJ5MVLYTQbeeUg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-spread": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-spread/-/plugin-transform-spread-7.16.7.tgz", + "integrity": "sha512-+pjJpgAngb53L0iaA5gU/1MLXJIfXcYepLgXB3esVRf4fqmj8f2cxM3/FKaHsZms08hFQJkFccEWuIpm429TXg==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-skip-transparent-expression-wrappers": "^7.16.0" + } + }, + "@babel/plugin-transform-sticky-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.16.7.tgz", + "integrity": "sha512-NJa0Bd/87QV5NZZzTuZG5BPJjLYadeSZ9fO6oOUoL4iQx+9EEuw/eEM92SrsT19Yc2jgB1u1hsjqDtH02c3Drw==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-template-literals": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.16.7.tgz", + "integrity": "sha512-VwbkDDUeenlIjmfNeDX/V0aWrQH2QiVyJtwymVQSzItFDTpxfyJh3EVaQiS0rIN/CqbLGr0VcGmuwyTdZtdIsA==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-typeof-symbol": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.16.7.tgz", + "integrity": "sha512-p2rOixCKRJzpg9JB4gjnG4gjWkWa89ZoYUnl9snJ1cWIcTH/hvxZqfO+WjG6T8DRBpctEol5jw1O5rA8gkCokQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-typescript": { + "version": "7.16.8", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.16.8.tgz", + "integrity": "sha512-bHdQ9k7YpBDO2d0NVfkj51DpQcvwIzIusJ7mEUaMlbZq3Kt/U47j24inXZHQ5MDiYpCs+oZiwnXyKedE8+q7AQ==", + "requires": { + "@babel/helper-create-class-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/plugin-syntax-typescript": "^7.16.7" + } + }, + "@babel/plugin-transform-unicode-escapes": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.16.7.tgz", + "integrity": "sha512-TAV5IGahIz3yZ9/Hfv35TV2xEm+kaBDaZQCn2S/hG9/CZ0DktxJv9eKfPc7yYCvOYR4JGx1h8C+jcSOvgaaI/Q==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/plugin-transform-unicode-regex": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.16.7.tgz", + "integrity": "sha512-oC5tYYKw56HO75KZVLQ+R/Nl3Hro9kf8iG0hXoaHP7tjAyCpvqBiSNe6vGrZni1Z6MggmUOC6A7VP7AVmw225Q==", + "requires": { + "@babel/helper-create-regexp-features-plugin": "^7.16.7", + "@babel/helper-plugin-utils": "^7.16.7" + } + }, + "@babel/preset-env": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/preset-env/-/preset-env-7.17.10.tgz", + "integrity": "sha512-YNgyBHZQpeoBSRBg0xixsZzfT58Ze1iZrajvv0lJc70qDDGuGfonEnMGfWeSY0mQ3JTuCWFbMkzFRVafOyJx4g==", + "requires": { + "@babel/compat-data": "^7.17.10", + "@babel/helper-compilation-targets": "^7.17.10", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression": "^7.16.7", + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining": "^7.16.7", + "@babel/plugin-proposal-async-generator-functions": "^7.16.8", + "@babel/plugin-proposal-class-properties": "^7.16.7", + "@babel/plugin-proposal-class-static-block": "^7.17.6", + "@babel/plugin-proposal-dynamic-import": "^7.16.7", + "@babel/plugin-proposal-export-namespace-from": "^7.16.7", + "@babel/plugin-proposal-json-strings": "^7.16.7", + "@babel/plugin-proposal-logical-assignment-operators": "^7.16.7", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.16.7", + "@babel/plugin-proposal-numeric-separator": "^7.16.7", + "@babel/plugin-proposal-object-rest-spread": "^7.17.3", + "@babel/plugin-proposal-optional-catch-binding": "^7.16.7", + "@babel/plugin-proposal-optional-chaining": "^7.16.7", + "@babel/plugin-proposal-private-methods": "^7.16.11", + "@babel/plugin-proposal-private-property-in-object": "^7.16.7", + "@babel/plugin-proposal-unicode-property-regex": "^7.16.7", + "@babel/plugin-syntax-async-generators": "^7.8.4", + "@babel/plugin-syntax-class-properties": "^7.12.13", + "@babel/plugin-syntax-class-static-block": "^7.14.5", + "@babel/plugin-syntax-dynamic-import": "^7.8.3", + "@babel/plugin-syntax-export-namespace-from": "^7.8.3", + "@babel/plugin-syntax-json-strings": "^7.8.3", + "@babel/plugin-syntax-logical-assignment-operators": "^7.10.4", + "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3", + "@babel/plugin-syntax-numeric-separator": "^7.10.4", + "@babel/plugin-syntax-object-rest-spread": "^7.8.3", + "@babel/plugin-syntax-optional-catch-binding": "^7.8.3", + "@babel/plugin-syntax-optional-chaining": "^7.8.3", + "@babel/plugin-syntax-private-property-in-object": "^7.14.5", + "@babel/plugin-syntax-top-level-await": "^7.14.5", + "@babel/plugin-transform-arrow-functions": "^7.16.7", + "@babel/plugin-transform-async-to-generator": "^7.16.8", + "@babel/plugin-transform-block-scoped-functions": "^7.16.7", + "@babel/plugin-transform-block-scoping": "^7.16.7", + "@babel/plugin-transform-classes": "^7.16.7", + "@babel/plugin-transform-computed-properties": "^7.16.7", + "@babel/plugin-transform-destructuring": "^7.17.7", + "@babel/plugin-transform-dotall-regex": "^7.16.7", + "@babel/plugin-transform-duplicate-keys": "^7.16.7", + "@babel/plugin-transform-exponentiation-operator": "^7.16.7", + "@babel/plugin-transform-for-of": "^7.16.7", + "@babel/plugin-transform-function-name": "^7.16.7", + "@babel/plugin-transform-literals": "^7.16.7", + "@babel/plugin-transform-member-expression-literals": "^7.16.7", + "@babel/plugin-transform-modules-amd": "^7.16.7", + "@babel/plugin-transform-modules-commonjs": "^7.17.9", + "@babel/plugin-transform-modules-systemjs": "^7.17.8", + "@babel/plugin-transform-modules-umd": "^7.16.7", + "@babel/plugin-transform-named-capturing-groups-regex": "^7.17.10", + "@babel/plugin-transform-new-target": "^7.16.7", + "@babel/plugin-transform-object-super": "^7.16.7", + "@babel/plugin-transform-parameters": "^7.16.7", + "@babel/plugin-transform-property-literals": "^7.16.7", + "@babel/plugin-transform-regenerator": "^7.17.9", + "@babel/plugin-transform-reserved-words": "^7.16.7", + "@babel/plugin-transform-shorthand-properties": "^7.16.7", + "@babel/plugin-transform-spread": "^7.16.7", + "@babel/plugin-transform-sticky-regex": "^7.16.7", + "@babel/plugin-transform-template-literals": "^7.16.7", + "@babel/plugin-transform-typeof-symbol": "^7.16.7", + "@babel/plugin-transform-unicode-escapes": "^7.16.7", + "@babel/plugin-transform-unicode-regex": "^7.16.7", + "@babel/preset-modules": "^0.1.5", + "@babel/types": "^7.17.10", + "babel-plugin-polyfill-corejs2": "^0.3.0", + "babel-plugin-polyfill-corejs3": "^0.5.0", + "babel-plugin-polyfill-regenerator": "^0.3.0", + "core-js-compat": "^3.22.1", + "semver": "^6.3.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "@babel/preset-modules": { + "version": "0.1.5", + "resolved": "https://registry.npmjs.org/@babel/preset-modules/-/preset-modules-0.1.5.tgz", + "integrity": "sha512-A57th6YRG7oR3cq/yt/Y84MvGgE0eJG2F1JLhKuyG+jFxEgrd/HAMJatiFtmOiZurz+0DkrvbheCLaV5f2JfjA==", + "requires": { + "@babel/helper-plugin-utils": "^7.0.0", + "@babel/plugin-proposal-unicode-property-regex": "^7.4.4", + "@babel/plugin-transform-dotall-regex": "^7.4.4", + "@babel/types": "^7.4.4", + "esutils": "^2.0.2" + } + }, + "@babel/preset-react": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/preset-react/-/preset-react-7.16.7.tgz", + "integrity": "sha512-fWpyI8UM/HE6DfPBzD8LnhQ/OcH8AgTaqcqP2nGOXEUV+VKBR5JRN9hCk9ai+zQQ57vtm9oWeXguBCPNUjytgA==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-transform-react-display-name": "^7.16.7", + "@babel/plugin-transform-react-jsx": "^7.16.7", + "@babel/plugin-transform-react-jsx-development": "^7.16.7", + "@babel/plugin-transform-react-pure-annotations": "^7.16.7" + } + }, + "@babel/preset-typescript": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/preset-typescript/-/preset-typescript-7.16.7.tgz", + "integrity": "sha512-WbVEmgXdIyvzB77AQjGBEyYPZx+8tTsO50XtfozQrkW8QB2rLJpH2lgx0TRw5EJrBxOZQ+wCcyPVQvS8tjEHpQ==", + "requires": { + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/helper-validator-option": "^7.16.7", + "@babel/plugin-transform-typescript": "^7.16.7" + } + }, + "@babel/runtime": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.17.9.tgz", + "integrity": "sha512-lSiBBvodq29uShpWGNbgFdKYNiFDo5/HIYsaCEY9ff4sb10x9jizo2+pRrSyF4jKZCXqgzuqBOQKbUm90gQwJg==", + "requires": { + "regenerator-runtime": "^0.13.4" + } + }, + "@babel/runtime-corejs3": { + "version": "7.17.9", + "resolved": "https://registry.npmjs.org/@babel/runtime-corejs3/-/runtime-corejs3-7.17.9.tgz", + "integrity": "sha512-WxYHHUWF2uZ7Hp1K+D1xQgbgkGUfA+5UPOegEXGt2Y5SMog/rYCVaifLZDbw8UkNXozEqqrZTy6bglL7xTaCOw==", + "requires": { + "core-js-pure": "^3.20.2", + "regenerator-runtime": "^0.13.4" + } + }, + "@babel/template": { + "version": "7.16.7", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.16.7.tgz", + "integrity": "sha512-I8j/x8kHUrbYRTUxXrrMbfCa7jxkE7tZre39x3kjr9hvI82cK1FfqLygotcWN5kdPGWcLdWMHpSBavse5tWw3w==", + "requires": { + "@babel/code-frame": "^7.16.7", + "@babel/parser": "^7.16.7", + "@babel/types": "^7.16.7" + } + }, + "@babel/traverse": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.17.10.tgz", + "integrity": "sha512-VmbrTHQteIdUUQNTb+zE12SHS/xQVIShmBPhlNP12hD5poF2pbITW1Z4172d03HegaQWhLffdkRJYtAzp0AGcw==", + "requires": { + "@babel/code-frame": "^7.16.7", + "@babel/generator": "^7.17.10", + "@babel/helper-environment-visitor": "^7.16.7", + "@babel/helper-function-name": "^7.17.9", + "@babel/helper-hoist-variables": "^7.16.7", + "@babel/helper-split-export-declaration": "^7.16.7", + "@babel/parser": "^7.17.10", + "@babel/types": "^7.17.10", + "debug": "^4.1.0", + "globals": "^11.1.0" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "@babel/types": { + "version": "7.17.10", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.17.10.tgz", + "integrity": "sha512-9O26jG0mBYfGkUYCYZRnBwbVLd1UZOICEr2Em6InB6jVfsAv1GKgwXHmrSg+WFWDmeKTA6vyTZiN8tCSM5Oo3A==", + "requires": { + "@babel/helper-validator-identifier": "^7.16.7", + "to-fast-properties": "^2.0.0" + } + }, + "@emotion/cache": { + "version": "11.7.1", + "resolved": "https://registry.npmmirror.com/@emotion/cache/-/cache-11.7.1.tgz", + "integrity": "sha512-r65Zy4Iljb8oyjtLeCuBH8Qjiy107dOYC6SJq7g7GV5UCQWMObY4SJDPGFjiiVpPrOJ2hmJOoBiYTC7hwx9E2A==", + "requires": { + "@emotion/memoize": "^0.7.4", + "@emotion/sheet": "^1.1.0", + "@emotion/utils": "^1.0.0", + "@emotion/weak-memoize": "^0.2.5", + "stylis": "4.0.13" + } + }, + "@emotion/hash": { + "version": "0.8.0", + "resolved": "https://registry.npmmirror.com/@emotion/hash/-/hash-0.8.0.tgz", + "integrity": "sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow==" + }, + "@emotion/memoize": { + "version": "0.7.5", + "resolved": "https://registry.npmmirror.com/@emotion/memoize/-/memoize-0.7.5.tgz", + "integrity": "sha512-igX9a37DR2ZPGYtV6suZ6whr8pTFtyHL3K/oLUotxpSVO2ASaprmAe2Dkq7tBo7CRY7MMDrAa9nuQP9/YG8FxQ==" + }, + "@emotion/react": { + "version": "11.7.1", + "resolved": "https://registry.npmmirror.com/@emotion/react/-/react-11.7.1.tgz", + "integrity": "sha512-DV2Xe3yhkF1yT4uAUoJcYL1AmrnO5SVsdfvu+fBuS7IbByDeTVx9+wFmvx9Idzv7/78+9Mgx2Hcmr7Fex3tIyw==", + "requires": { + "@babel/runtime": "^7.13.10", + "@emotion/cache": "^11.7.1", + "@emotion/serialize": "^1.0.2", + "@emotion/sheet": "^1.1.0", + "@emotion/utils": "^1.0.0", + "@emotion/weak-memoize": "^0.2.5", + "hoist-non-react-statics": "^3.3.1" + } + }, + "@emotion/serialize": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/@emotion/serialize/-/serialize-1.0.2.tgz", + "integrity": "sha512-95MgNJ9+/ajxU7QIAruiOAdYNjxZX7G2mhgrtDWswA21VviYIRP1R5QilZ/bDY42xiKsaktP4egJb3QdYQZi1A==", + "requires": { + "@emotion/hash": "^0.8.0", + "@emotion/memoize": "^0.7.4", + "@emotion/unitless": "^0.7.5", + "@emotion/utils": "^1.0.0", + "csstype": "^3.0.2" + } + }, + "@emotion/server": { + "version": "11.4.0", + "resolved": "https://registry.npmmirror.com/@emotion/server/-/server-11.4.0.tgz", + "integrity": "sha512-IHovdWA3V0DokzxLtUNDx4+hQI82zUXqQFcVz/om2t44O0YSc+NHB+qifnyAOoQwt3SXcBTgaSntobwUI9gnfA==", + "requires": { + "@emotion/utils": "^1.0.0", + "html-tokenize": "^2.0.0", + "multipipe": "^1.0.2", + "through": "^2.3.8" + } + }, + "@emotion/sheet": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/@emotion/sheet/-/sheet-1.1.0.tgz", + "integrity": "sha512-u0AX4aSo25sMAygCuQTzS+HsImZFuS8llY8O7b9MDRzbJM0kVJlAz6KNDqcG7pOuQZJmj/8X/rAW+66kMnMW+g==" + }, + "@emotion/unitless": { + "version": "0.7.5", + "resolved": "https://registry.npmmirror.com/@emotion/unitless/-/unitless-0.7.5.tgz", + "integrity": "sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg==" + }, + "@emotion/utils": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/@emotion/utils/-/utils-1.0.0.tgz", + "integrity": "sha512-mQC2b3XLDs6QCW+pDQDiyO/EdGZYOygE8s5N5rrzjSI4M3IejPE/JPndCBwRT9z982aqQNi6beWs1UeayrQxxA==" + }, + "@emotion/weak-memoize": { + "version": "0.2.5", + "resolved": "https://registry.npmmirror.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz", + "integrity": "sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA==" + }, + "@endemolshinegroup/cosmiconfig-typescript-loader": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/@endemolshinegroup/cosmiconfig-typescript-loader/-/cosmiconfig-typescript-loader-3.0.2.tgz", + "integrity": "sha512-QRVtqJuS1mcT56oHpVegkKBlgtWjXw/gHNWO3eL9oyB5Sc7HBoc2OLG/nYpVfT/Jejvo3NUrD0Udk7XgoyDKkA==", + "requires": { + "lodash.get": "^4", + "make-error": "^1", + "ts-node": "^9", + "tslib": "^2" + } + }, + "@eslint/eslintrc": { + "version": "0.4.3", + "resolved": "https://registry.npmjs.org/@eslint/eslintrc/-/eslintrc-0.4.3.tgz", + "integrity": "sha512-J6KFFz5QCYUJq3pf0mjEcCJVERbzv71PUIDczuh9JkwGEzced6CO5ADLHB1rbf/+oPBtoPfMYNOpGDzCANlbXw==", + "requires": { + "ajv": "^6.12.4", + "debug": "^4.1.1", + "espree": "^7.3.0", + "globals": "^13.9.0", + "ignore": "^4.0.6", + "import-fresh": "^3.2.1", + "js-yaml": "^3.13.1", + "minimatch": "^3.0.4", + "strip-json-comments": "^3.1.1" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + }, + "globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "requires": { + "type-fest": "^0.20.2" + } + }, + "ignore": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-4.0.6.tgz", + "integrity": "sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg==" + } + } + }, + "@gatsbyjs/parcel-namer-relative-to-cwd": { + "version": "0.0.2", + "resolved": "https://registry.npmjs.org/@gatsbyjs/parcel-namer-relative-to-cwd/-/parcel-namer-relative-to-cwd-0.0.2.tgz", + "integrity": "sha512-ZeGxCbx13+zjpE/0HuJ/tjox9zfiYq9fGoAAi+RHP5vHSJCmJVO5hZbexQ/umlUyAkkkzC4p1WIpw1cYQTA8SA==", + "requires": { + "@babel/runtime": "^7.15.4", + "@parcel/plugin": "2.3.1", + "gatsby-core-utils": "^3.8.2" + }, + "dependencies": { + "@parcel/cache": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/cache/-/cache-2.3.1.tgz", + "integrity": "sha512-8Wvm0VERtocUepIfkZ6xVs1LHZqttnzdrM7oSc0bXhwtz8kZB++N88g0rQskbUchW87314eYdzBtEL0aiq0bgQ==", + "requires": { + "@parcel/fs": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/utils": "2.3.1", + "lmdb": "^2.0.2" + } + }, + "@parcel/codeframe": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/codeframe/-/codeframe-2.3.1.tgz", + "integrity": "sha512-sdNvbg9qYS2pwzqyyyt+wZfNGuy7EslzDLbzQclFZmhD6e770mcYoi8/7i7D/AONbXiI15vwNmgOdcUIXtPxbA==", + "requires": { + "chalk": "^4.1.0" + } + }, + "@parcel/diagnostic": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/diagnostic/-/diagnostic-2.3.1.tgz", + "integrity": "sha512-hBMcg4WVMdSIy6RpI4gSto5dZ3OoUbnrCZzVw3J1tzQJn7x9na/+014IaE58vJtAqJ8/jc/TqWIcwsSLe898rA==", + "requires": { + "json-source-map": "^0.6.1", + "nullthrows": "^1.1.1" + } + }, + "@parcel/events": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/events/-/events-2.3.1.tgz", + "integrity": "sha512-J2rWKGl1Z2IvwwDwWYz/4gUxC1P4LsioUyOo1HYGT+N5+r41P8ZB5CM/aosI2qu5mMsH8rTpclOv5E36vCSQxw==" + }, + "@parcel/fs": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/fs/-/fs-2.3.1.tgz", + "integrity": "sha512-FKqyf8KF0zOw8gfj/feEAMj4Kzqkgt9Zxa2A7UDdMWRvxLR8znqnWjD++xqq6rxJp2Y1zm4fH3JOTK4CRddUSg==", + "requires": { + "@parcel/fs-search": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "@parcel/watcher": "^2.0.0", + "@parcel/workers": "2.3.1" + } + }, + "@parcel/fs-search": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/fs-search/-/fs-search-2.3.1.tgz", + "integrity": "sha512-JsBIDttjmgJIMD6Q6MV83M+mwr5NqUm55iA+SewimboiWzSPzIJxRaegniSsNfsrBASJ6nSZFHcLPd/VJ5iqJw==", + "requires": { + "detect-libc": "^1.0.3" + } + }, + "@parcel/hash": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/hash/-/hash-2.3.1.tgz", + "integrity": "sha512-IYhSQE+CIKWjPfiLmsrXHupkNd+hMlTlI9DR5qLiD8ydyPwg0XE/bOYTcbdsSl6HTackY0XYVSJwTtEgvtYVfw==", + "requires": { + "detect-libc": "^1.0.3", + "xxhash-wasm": "^0.4.2" + } + }, + "@parcel/logger": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/logger/-/logger-2.3.1.tgz", + "integrity": "sha512-swNPInULCJrpCJCLOgZcf+xNcUF0NjD7LyNcB349BkyO7i6st14nfBjXf6eAJJu0z7RMmi6zp9CQB47e4cI6+g==", + "requires": { + "@parcel/diagnostic": "2.3.1", + "@parcel/events": "2.3.1" + } + }, + "@parcel/markdown-ansi": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/markdown-ansi/-/markdown-ansi-2.3.1.tgz", + "integrity": "sha512-M4Hi25pKtSh1KF/ppMDBk5QuLpYAQjgB/MSP+nz7NzXQlYPCN5oEk9TUkrmQ9J+vOvVwefxfy7ahSErEuQbTFw==", + "requires": { + "chalk": "^4.1.0" + } + }, + "@parcel/package-manager": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/package-manager/-/package-manager-2.3.1.tgz", + "integrity": "sha512-w2XOkD3SU8RxhUDW+Soy/TjvEVvfUsBmHy02asllt4b/ZtyZVAsQmonGExHDDkRn3TNDR6Y96Yw6M7purt+b9w==", + "requires": { + "@parcel/diagnostic": "2.3.1", + "@parcel/fs": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "@parcel/workers": "2.3.1", + "semver": "^5.7.1" + } + }, + "@parcel/plugin": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/plugin/-/plugin-2.3.1.tgz", + "integrity": "sha512-ROOWbgFze7BCF3RkEh8VbcKGlR5UGBuJ8lfCaFrG1VOk7Rxgl8Bmk96TRbZREm/1jB74p2O8twVKyPSC13riow==", + "requires": { + "@parcel/types": "2.3.1" + } + }, + "@parcel/types": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/types/-/types-2.3.1.tgz", + "integrity": "sha512-i2UyUoA4DzyYxe9rZRDuMAZ6TD3Mq3tTTqeJ2/zA6w83Aon3cqdE9va91peu1fKRGyRqE5lwWRtA7ktF1A2SVA==", + "requires": { + "@parcel/cache": "2.3.1", + "@parcel/diagnostic": "2.3.1", + "@parcel/fs": "2.3.1", + "@parcel/package-manager": "2.3.1", + "@parcel/source-map": "^2.0.0", + "@parcel/workers": "2.3.1", + "utility-types": "^3.10.0" + } + }, + "@parcel/utils": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/utils/-/utils-2.3.1.tgz", + "integrity": "sha512-OFdh/HuAcce753/U3QoORzYU3N5oZqCfQNRb0i3onuz/qpli5TyxUl/k1BuTqlKYr6Px3kj05g6GFi9kRBOMbw==", + "requires": { + "@parcel/codeframe": "2.3.1", + "@parcel/diagnostic": "2.3.1", + "@parcel/hash": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/markdown-ansi": "2.3.1", + "@parcel/source-map": "^2.0.0", + "chalk": "^4.1.0" + } + }, + "@parcel/workers": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/@parcel/workers/-/workers-2.3.1.tgz", + "integrity": "sha512-e2P/9p5AYBLfNRs8n+57ChGrn5171oHwY54dz/jj0CrXKN1q0b+rNwzYsPaAtOicBoqmm1s5I3cjfO6GfJP65A==", + "requires": { + "@parcel/diagnostic": "2.3.1", + "@parcel/logger": "2.3.1", + "@parcel/types": "2.3.1", + "@parcel/utils": "2.3.1", + "chrome-trace-event": "^1.0.2", + "nullthrows": "^1.1.1" + } + }, + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + } + } + }, + "@gatsbyjs/potrace": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@gatsbyjs/potrace/-/potrace-2.2.0.tgz", + "integrity": "sha512-/RiLVFJA+CIYhceb6XL1kD1GZ5E2JBX38pld0fiGNiNwLl+Bb7TYZR72aQvcs3v+NOrSjbagUiCnIHYmEW4F7w==", + "requires": { + "jimp": "^0.16.1" + } + }, + "@gatsbyjs/reach-router": { + "version": "1.3.7", + "resolved": "https://registry.npmjs.org/@gatsbyjs/reach-router/-/reach-router-1.3.7.tgz", + "integrity": "sha512-KQ5FvMb4BZUlSo+yQgd4t4WB8vkVPWfKjTpSl+Bx/FZhU6OL4lpwgfX7fXAY/18DogqyJCFiNAjV5eo3rQ5Alw==", + "requires": { + "invariant": "^2.2.3", + "prop-types": "^15.6.1", + "react-lifecycles-compat": "^3.0.4" + } + }, + "@gatsbyjs/webpack-hot-middleware": { + "version": "2.25.3", + "resolved": "https://registry.npmjs.org/@gatsbyjs/webpack-hot-middleware/-/webpack-hot-middleware-2.25.3.tgz", + "integrity": "sha512-ul17OZ8Dlw+ATRbnuU+kwxuAlq9lKbYz/2uBS1FLCdgoPTF1H2heP7HbUbgfMZbfRQNcCG2rMscMnr32ritCDw==", + "requires": { + "ansi-html-community": "0.0.8", + "html-entities": "^2.3.3", + "strip-ansi": "^6.0.0" + } + }, + "@graphql-codegen/add": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/add/-/add-3.1.1.tgz", + "integrity": "sha512-XkVwcqosa0CVBlL1HaQT0gp+EUfhuQE3LzrEpzMQLwchxaj/NPVYtOJL6MUHaYDsHzLqxWrufjfbeB3y2NQgRw==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.3.2", + "tslib": "~2.3.0" + } + }, + "@graphql-codegen/core": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/core/-/core-2.5.1.tgz", + "integrity": "sha512-alctBVl2hMnBXDLwkgmnFPrZVIiBDsWJSmxJcM4GKg1PB23+xuov35GE47YAyAhQItE1B1fbYnbb1PtGiDZ4LA==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.4.1", + "@graphql-tools/schema": "^8.1.2", + "@graphql-tools/utils": "^8.1.1", + "tslib": "~2.3.0" + } + }, + "@graphql-codegen/plugin-helpers": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/@graphql-codegen/plugin-helpers/-/plugin-helpers-2.4.2.tgz", + "integrity": "sha512-LJNvwAPv/sKtI3RnRDm+nPD+JeOfOuSOS4FFIpQCMUCyMnFcchV/CPTTv7tT12fLUpEg6XjuFfDBvOwndti30Q==", + "requires": { + "@graphql-tools/utils": "^8.5.2", + "change-case-all": "1.0.14", + "common-tags": "1.8.2", + "import-from": "4.0.0", + "lodash": "~4.17.0", + "tslib": "~2.3.0" + } + }, + "@graphql-codegen/schema-ast": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/@graphql-codegen/schema-ast/-/schema-ast-2.4.1.tgz", + "integrity": "sha512-bIWlKk/ShoVJfghA4Rt1OWnd34/dQmZM/vAe6fu6QKyOh44aAdqPtYQ2dbTyFXoknmu504etKJGEDllYNUJRfg==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.3.2", + "@graphql-tools/utils": "^8.1.1", + "tslib": "~2.3.0" + } + }, + "@graphql-codegen/typescript": { + "version": "2.4.11", + "resolved": "https://registry.npmjs.org/@graphql-codegen/typescript/-/typescript-2.4.11.tgz", + "integrity": "sha512-K3oDLPJRH9Wgpg9TOvb7L+xrJZ8HxkIzV2umqGn54c+8DQjvnRFBIYRO0THgUBMnEauE2sEy6RZkGHGfgQUruA==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-codegen/schema-ast": "^2.4.1", + "@graphql-codegen/visitor-plugin-common": "2.8.0", + "auto-bind": "~4.0.0", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-codegen/typescript-operations": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/@graphql-codegen/typescript-operations/-/typescript-operations-2.4.0.tgz", + "integrity": "sha512-vJ15FLyWchuO2Xkp6uz7jJOdChiay7P9KJKFDILx/JTwjinU1fFa7iOvyeTvslqiUPxgsXthR5izdY+E5IyLkQ==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-codegen/typescript": "^2.4.11", + "@graphql-codegen/visitor-plugin-common": "2.8.0", + "auto-bind": "~4.0.0", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-codegen/visitor-plugin-common": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/@graphql-codegen/visitor-plugin-common/-/visitor-plugin-common-2.8.0.tgz", + "integrity": "sha512-29MOaxBog7qaEhmeCzJn2mONSbcA+slCTzHN4nJ3aZl4KrC9V32rXlQpG5x0qHbFQ1LaG1f5gPO83xbiAeMBIw==", + "requires": { + "@graphql-codegen/plugin-helpers": "^2.4.0", + "@graphql-tools/optimize": "^1.0.1", + "@graphql-tools/relay-operation-optimizer": "^6.3.7", + "@graphql-tools/utils": "^8.3.0", + "auto-bind": "~4.0.0", + "change-case-all": "1.0.14", + "dependency-graph": "^0.11.0", + "graphql-tag": "^2.11.0", + "parse-filepath": "^1.0.2", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/batch-execute": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/@graphql-tools/batch-execute/-/batch-execute-7.1.2.tgz", + "integrity": "sha512-IuR2SB2MnC2ztA/XeTMTfWcA0Wy7ZH5u+nDkDNLAdX+AaSyDnsQS35sCmHqG0VOGTl7rzoyBWLCKGwSJplgtwg==", + "requires": { + "@graphql-tools/utils": "^7.7.0", + "dataloader": "2.0.0", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "dependencies": { + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + } + }, + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==" + } + } + }, + "@graphql-tools/code-file-loader": { + "version": "7.2.15", + "resolved": "https://registry.npmjs.org/@graphql-tools/code-file-loader/-/code-file-loader-7.2.15.tgz", + "integrity": "sha512-tqViC0MEXadcGofqDU4mL7eA/+TyM4iCP+XRkL/mdAMYWp1JdzMYUJcCH47ZK1+EvNctYx8sJOpSGlvLR4ZFFg==", + "requires": { + "@graphql-tools/graphql-tag-pluck": "7.2.7", + "@graphql-tools/utils": "8.6.10", + "globby": "^11.0.3", + "tslib": "~2.4.0", + "unixify": "^1.0.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/delegate": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/delegate/-/delegate-7.1.5.tgz", + "integrity": "sha512-bQu+hDd37e+FZ0CQGEEczmRSfQRnnXeUxI/0miDV+NV/zCbEdIJj5tYFNrKT03W6wgdqx8U06d8L23LxvGri/g==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "@graphql-tools/batch-execute": "^7.1.2", + "@graphql-tools/schema": "^7.1.5", + "@graphql-tools/utils": "^7.7.1", + "dataloader": "2.0.0", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "dependencies": { + "@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "requires": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + } + }, + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + } + }, + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==" + } + } + }, + "@graphql-tools/graphql-file-loader": { + "version": "6.2.7", + "resolved": "https://registry.npmjs.org/@graphql-tools/graphql-file-loader/-/graphql-file-loader-6.2.7.tgz", + "integrity": "sha512-5k2SNz0W87tDcymhEMZMkd6/vs6QawDyjQXWtqkuLTBF3vxjxPD1I4dwHoxgWPIjjANhXybvulD7E+St/7s9TQ==", + "requires": { + "@graphql-tools/import": "^6.2.6", + "@graphql-tools/utils": "^7.0.0", + "tslib": "~2.1.0" + }, + "dependencies": { + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "dependencies": { + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + } + } + }, + "tslib": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.1.0.tgz", + "integrity": "sha512-hcVC3wYEziELGGmEEXue7D75zbwIIVUMWAVbHItGPx0ziyXxrOMQx4rQEVEV45Ut/1IotuEvwqPopzIOkDMf0A==" + } + } + }, + "@graphql-tools/graphql-tag-pluck": { + "version": "7.2.7", + "resolved": "https://registry.npmjs.org/@graphql-tools/graphql-tag-pluck/-/graphql-tag-pluck-7.2.7.tgz", + "integrity": "sha512-leh6rVHNoGXjmbOtTaKNXQtFw4Gu7PokTlEOweNKYMssbH6L1zrKA0G4cE55s60JBo9SF4hjg8X7Gkech5t+mQ==", + "requires": { + "@babel/parser": "^7.16.8", + "@babel/traverse": "^7.16.8", + "@babel/types": "^7.16.8", + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/import": { + "version": "6.6.14", + "resolved": "https://registry.npmjs.org/@graphql-tools/import/-/import-6.6.14.tgz", + "integrity": "sha512-XN6swtMdUxd9czmdNIK6yJ0w5t4FOUWSoSkYP0+to8j44r8zdM3nsAppoA0OLmsUY+JnTBgkW3jGlOFvqC3HWg==", + "requires": { + "@graphql-tools/utils": "8.6.10", + "resolve-from": "5.0.0", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/json-file-loader": { + "version": "6.2.6", + "resolved": "https://registry.npmjs.org/@graphql-tools/json-file-loader/-/json-file-loader-6.2.6.tgz", + "integrity": "sha512-CnfwBSY5926zyb6fkDBHnlTblHnHI4hoBALFYXnrg0Ev4yWU8B04DZl/pBRUc459VNgO2x8/mxGIZj2hPJG1EA==", + "requires": { + "@graphql-tools/utils": "^7.0.0", + "tslib": "~2.0.1" + }, + "dependencies": { + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + }, + "dependencies": { + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + } + } + }, + "tslib": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.0.3.tgz", + "integrity": "sha512-uZtkfKblCEQtZKBF6EBXVZeQNl82yqtDQdv+eck8u7tdPxjLu2/lp5/uPW+um2tpuxINHWy3GhiccY7QgEaVHQ==" + } + } + }, + "@graphql-tools/load": { + "version": "7.5.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/load/-/load-7.5.11.tgz", + "integrity": "sha512-a8sD3iHfxcbIwP0nSxF+DUAVg+/MuLNOizVJHcZGGS8AdDoezUsnWRkNDT6FlVqRoxHNbkpq8+6B55JKtqHSxg==", + "requires": { + "@graphql-tools/schema": "8.3.11", + "@graphql-tools/utils": "8.6.10", + "p-limit": "3.1.0", + "tslib": "~2.4.0" + }, + "dependencies": { + "p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "requires": { + "yocto-queue": "^0.1.0" + } + }, + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/merge": { + "version": "8.2.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/merge/-/merge-8.2.11.tgz", + "integrity": "sha512-fsjJVdsk9GV1jj1Ed2AKLlHYlsf0ZadTK8X5KxFRE1ZSnKqh56BLVX93JrtOIAnsiHkwOK2TC43HGhApF1swpQ==", + "requires": { + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/optimize": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/optimize/-/optimize-1.2.0.tgz", + "integrity": "sha512-l0PTqgHeorQdeOizUor6RB49eOAng9+abSxiC5/aHRo6hMmXVaqv5eqndlmxCpx9BkgNb3URQbK+ZZHVktkP/g==", + "requires": { + "tslib": "~2.3.0" + } + }, + "@graphql-tools/relay-operation-optimizer": { + "version": "6.4.10", + "resolved": "https://registry.npmjs.org/@graphql-tools/relay-operation-optimizer/-/relay-operation-optimizer-6.4.10.tgz", + "integrity": "sha512-a5wDdXP7MmwZDy9R8+RZ0ajJBWX1Lk9sIG6uSIo5G/LnGpXncgBhKpJf5r6rOf0zsFLWnAkYm/dCDMpFaGE/Yw==", + "requires": { + "@graphql-tools/utils": "8.6.10", + "relay-compiler": "12.0.0", + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/schema": { + "version": "8.3.11", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-8.3.11.tgz", + "integrity": "sha512-esMEnbyXbp8B5VEI4o395+x0G7Qmz3JSX5onFBF8HeLYcqWJasY5vBuWkO18VxrZpEnvnryodP6Y00bVag9O3Q==", + "requires": { + "@graphql-tools/merge": "8.2.11", + "@graphql-tools/utils": "8.6.10", + "tslib": "~2.4.0", + "value-or-promise": "1.0.11" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/url-loader": { + "version": "6.10.1", + "resolved": "https://registry.npmjs.org/@graphql-tools/url-loader/-/url-loader-6.10.1.tgz", + "integrity": "sha512-DSDrbhQIv7fheQ60pfDpGD256ixUQIR6Hhf9Z5bRjVkXOCvO5XrkwoWLiU7iHL81GB1r0Ba31bf+sl+D4nyyfw==", + "requires": { + "@graphql-tools/delegate": "^7.0.1", + "@graphql-tools/utils": "^7.9.0", + "@graphql-tools/wrap": "^7.0.4", + "@microsoft/fetch-event-source": "2.0.1", + "@types/websocket": "1.0.2", + "abort-controller": "3.0.0", + "cross-fetch": "3.1.4", + "extract-files": "9.0.0", + "form-data": "4.0.0", + "graphql-ws": "^4.4.1", + "is-promise": "4.0.0", + "isomorphic-ws": "4.0.1", + "lodash": "4.17.21", + "meros": "1.1.4", + "subscriptions-transport-ws": "^0.9.18", + "sync-fetch": "0.3.0", + "tslib": "~2.2.0", + "valid-url": "1.0.9", + "ws": "7.4.5" + }, + "dependencies": { + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + } + }, + "cross-fetch": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/cross-fetch/-/cross-fetch-3.1.4.tgz", + "integrity": "sha512-1eAtFWdIubi6T4XPy6ei9iUFoKpUkIF971QLN8lIvvvwueI65+Nw5haMNKUwfJxabqlIIDODJKGrQ66gxC0PbQ==", + "requires": { + "node-fetch": "2.6.1" + } + }, + "node-fetch": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.1.tgz", + "integrity": "sha512-V4aYg89jEoVRxRb2fJdAg8FHvI7cEyYdVAh94HH0UIK8oJxUfkjlDQN9RbMx+bEjP7+ggMiFRprSti032Oipxw==" + }, + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + } + } + }, + "@graphql-tools/utils": { + "version": "8.6.10", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-8.6.10.tgz", + "integrity": "sha512-bJH9qwuyM3BP0PTU6/lvBDkk6jdEIOn+dbyk4pHMVNnvbJ1gZQwo62To8SHxxaUTus8OMhhVPSh9ApWXREURcg==", + "requires": { + "tslib": "~2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@graphql-tools/wrap": { + "version": "7.0.8", + "resolved": "https://registry.npmjs.org/@graphql-tools/wrap/-/wrap-7.0.8.tgz", + "integrity": "sha512-1NDUymworsOlb53Qfh7fonDi2STvqCtbeE68ntKY9K/Ju/be2ZNxrFSbrBHwnxWcN9PjISNnLcAyJ1L5tCUyhg==", + "requires": { + "@graphql-tools/delegate": "^7.1.5", + "@graphql-tools/schema": "^7.1.5", + "@graphql-tools/utils": "^7.8.1", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + }, + "dependencies": { + "@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "requires": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + } + }, + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + } + }, + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==" + } + } + }, + "@hapi/address": { + "version": "2.1.4", + "resolved": "https://registry.npmjs.org/@hapi/address/-/address-2.1.4.tgz", + "integrity": "sha512-QD1PhQk+s31P1ixsX0H0Suoupp3VMXzIVMSwobR3F3MSUO2YCV0B7xqLcUw/Bh8yuvd3LhpyqLQWTNcRmp6IdQ==" + }, + "@hapi/bourne": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/@hapi/bourne/-/bourne-1.3.2.tgz", + "integrity": "sha512-1dVNHT76Uu5N3eJNTYcvxee+jzX4Z9lfciqRRHCU27ihbUcYi+iSc2iml5Ke1LXe1SyJCLA0+14Jh4tXJgOppA==" + }, + "@hapi/hoek": { + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-9.3.0.tgz", + "integrity": "sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ==" + }, + "@hapi/joi": { + "version": "15.1.1", + "resolved": "https://registry.npmjs.org/@hapi/joi/-/joi-15.1.1.tgz", + "integrity": "sha512-entf8ZMOK8sc+8YfeOlM8pCfg3b5+WZIKBfUaaJT8UsjAAPjartzxIYm3TIbjvA4u+u++KbcXD38k682nVHDAQ==", + "requires": { + "@hapi/address": "2.x.x", + "@hapi/bourne": "1.x.x", + "@hapi/hoek": "8.x.x", + "@hapi/topo": "3.x.x" + }, + "dependencies": { + "@hapi/hoek": { + "version": "8.5.1", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-8.5.1.tgz", + "integrity": "sha512-yN7kbciD87WzLGc5539Tn0sApjyiGHAJgKvG9W8C7O+6c7qmoQMfVs0W4bX17eqz6C78QJqqFrtgdK5EWf6Qow==" + }, + "@hapi/topo": { + "version": "3.1.6", + "resolved": "https://registry.npmjs.org/@hapi/topo/-/topo-3.1.6.tgz", + "integrity": "sha512-tAag0jEcjwH+P2quUfipd7liWCNX2F8NvYjQp2wtInsZxnMlypdw0FtAOLxtvvkO+GSRRbmNi8m/5y42PQJYCQ==", + "requires": { + "@hapi/hoek": "^8.3.0" + } + } + } + }, + "@hapi/topo": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/@hapi/topo/-/topo-5.1.0.tgz", + "integrity": "sha512-foQZKJig7Ob0BMAYBfcJk8d77QtOe7Wo4ox7ff1lQYoNNAb6jwcY1ncdoy2e9wQZzvNy7ODZCYJkK8kzmcAnAg==", + "requires": { + "@hapi/hoek": "^9.0.0" + } + }, + "@humanwhocodes/config-array": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.5.0.tgz", + "integrity": "sha512-FagtKFz74XrTl7y6HCzQpwDfXP0yhxe9lHLD1UZxjvZIcbyRz8zTFF/yYNfSfzU414eDwZ1SrO0Qvtyf+wFMQg==", + "requires": { + "@humanwhocodes/object-schema": "^1.2.0", + "debug": "^4.1.1", + "minimatch": "^3.0.4" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "@humanwhocodes/object-schema": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz", + "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==" + }, + "@iarna/toml": { + "version": "2.2.5", + "resolved": "https://registry.npmjs.org/@iarna/toml/-/toml-2.2.5.tgz", + "integrity": "sha512-trnsAYxU3xnS1gPHPyU961coFyLkh4gAD/0zQ5mymY4yOZ+CYvsPqUbOFSw0aDM4y0tV7tiFxL/1XfXPNC6IPg==" + }, + "@icon-park/react": { + "version": "1.3.5", + "resolved": "https://registry.npmmirror.com/@icon-park/react/-/react-1.3.5.tgz", + "integrity": "sha512-FL3+BcYfFbRRl6DA6V2yA/QEoRRrhaDvMXLbFz0uW9lIsstQB6au/8aQxthB1G+eiqLxcw6yVULbmbBySn4Paw==", + "requires": {} + }, + "@jimp/bmp": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/bmp/-/bmp-0.16.1.tgz", + "integrity": "sha512-iwyNYQeBawrdg/f24x3pQ5rEx+/GwjZcCXd3Kgc+ZUd+Ivia7sIqBsOnDaMZdKCBPlfW364ekexnlOqyVa0NWg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "bmp-js": "^0.1.0" + } + }, + "@jimp/core": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/core/-/core-0.16.1.tgz", + "integrity": "sha512-la7kQia31V6kQ4q1kI/uLimu8FXx7imWVajDGtwUG8fzePLWDFJyZl0fdIXVCL1JW2nBcRHidUot6jvlRDi2+g==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "any-base": "^1.1.0", + "buffer": "^5.2.0", + "exif-parser": "^0.1.12", + "file-type": "^9.0.0", + "load-bmfont": "^1.3.1", + "mkdirp": "^0.5.1", + "phin": "^2.9.1", + "pixelmatch": "^4.0.2", + "tinycolor2": "^1.4.1" + }, + "dependencies": { + "file-type": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/file-type/-/file-type-9.0.0.tgz", + "integrity": "sha512-Qe/5NJrgIOlwijpq3B7BEpzPFcgzggOTagZmkXQY4LA6bsXKTUstK7Wp12lEJ/mLKTpvIZxmIuRcLYWT6ov9lw==" + } + } + }, + "@jimp/custom": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/custom/-/custom-0.16.1.tgz", + "integrity": "sha512-DNUAHNSiUI/j9hmbatD6WN/EBIyeq4AO0frl5ETtt51VN1SvE4t4v83ZA/V6ikxEf3hxLju4tQ5Pc3zmZkN/3A==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/core": "^0.16.1" + } + }, + "@jimp/gif": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/gif/-/gif-0.16.1.tgz", + "integrity": "sha512-r/1+GzIW1D5zrP4tNrfW+3y4vqD935WBXSc8X/wm23QTY9aJO9Lw6PEdzpYCEY+SOklIFKaJYUAq/Nvgm/9ryw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "gifwrap": "^0.9.2", + "omggif": "^1.0.9" + } + }, + "@jimp/jpeg": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/jpeg/-/jpeg-0.16.1.tgz", + "integrity": "sha512-8352zrdlCCLFdZ/J+JjBslDvml+fS3Z8gttdml0We759PnnZGqrnPRhkOEOJbNUlE+dD4ckLeIe6NPxlS/7U+w==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "jpeg-js": "0.4.2" + } + }, + "@jimp/plugin-blit": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-blit/-/plugin-blit-0.16.1.tgz", + "integrity": "sha512-fKFNARm32RoLSokJ8WZXHHH2CGzz6ire2n1Jh6u+XQLhk9TweT1DcLHIXwQMh8oR12KgjbgsMGvrMVlVknmOAg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-blur": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-blur/-/plugin-blur-0.16.1.tgz", + "integrity": "sha512-1WhuLGGj9MypFKRcPvmW45ht7nXkOKu+lg3n2VBzIB7r4kKNVchuI59bXaCYQumOLEqVK7JdB4glaDAbCQCLyw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-circle": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-circle/-/plugin-circle-0.16.1.tgz", + "integrity": "sha512-JK7yi1CIU7/XL8hdahjcbGA3V7c+F+Iw+mhMQhLEi7Q0tCnZ69YJBTamMiNg3fWPVfMuvWJJKOBRVpwNTuaZRg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-color": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-color/-/plugin-color-0.16.1.tgz", + "integrity": "sha512-9yQttBAO5SEFj7S6nJK54f+1BnuBG4c28q+iyzm1JjtnehjqMg6Ljw4gCSDCvoCQ3jBSYHN66pmwTV74SU1B7A==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "tinycolor2": "^1.4.1" + } + }, + "@jimp/plugin-contain": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-contain/-/plugin-contain-0.16.1.tgz", + "integrity": "sha512-44F3dUIjBDHN+Ym/vEfg+jtjMjAqd2uw9nssN67/n4FdpuZUVs7E7wadKY1RRNuJO+WgcD5aDQcsvurXMETQTg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-cover": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-cover/-/plugin-cover-0.16.1.tgz", + "integrity": "sha512-YztWCIldBAVo0zxcQXR+a/uk3/TtYnpKU2CanOPJ7baIuDlWPsG+YE4xTsswZZc12H9Kl7CiziEbDtvF9kwA/Q==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-crop": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-crop/-/plugin-crop-0.16.1.tgz", + "integrity": "sha512-UQdva9oQzCVadkyo3T5Tv2CUZbf0klm2cD4cWMlASuTOYgaGaFHhT9st+kmfvXjKL8q3STkBu/zUPV6PbuV3ew==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-displace": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-displace/-/plugin-displace-0.16.1.tgz", + "integrity": "sha512-iVAWuz2+G6Heu8gVZksUz+4hQYpR4R0R/RtBzpWEl8ItBe7O6QjORAkhxzg+WdYLL2A/Yd4ekTpvK0/qW8hTVw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-dither": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-dither/-/plugin-dither-0.16.1.tgz", + "integrity": "sha512-tADKVd+HDC9EhJRUDwMvzBXPz4GLoU6s5P7xkVq46tskExYSptgj5713J5Thj3NMgH9Rsqu22jNg1H/7tr3V9Q==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-fisheye": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-fisheye/-/plugin-fisheye-0.16.1.tgz", + "integrity": "sha512-BWHnc5hVobviTyIRHhIy9VxI1ACf4CeSuCfURB6JZm87YuyvgQh5aX5UDKtOz/3haMHXBLP61ZBxlNpMD8CG4A==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-flip": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-flip/-/plugin-flip-0.16.1.tgz", + "integrity": "sha512-KdxTf0zErfZ8DyHkImDTnQBuHby+a5YFdoKI/G3GpBl3qxLBvC+PWkS2F/iN3H7wszP7/TKxTEvWL927pypT0w==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-gaussian": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-gaussian/-/plugin-gaussian-0.16.1.tgz", + "integrity": "sha512-u9n4wjskh3N1mSqketbL6tVcLU2S5TEaFPR40K6TDv4phPLZALi1Of7reUmYpVm8mBDHt1I6kGhuCJiWvzfGyg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-invert": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-invert/-/plugin-invert-0.16.1.tgz", + "integrity": "sha512-2DKuyVXANH8WDpW9NG+PYFbehzJfweZszFYyxcaewaPLN0GxvxVLOGOPP1NuUTcHkOdMFbE0nHDuB7f+sYF/2w==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-mask": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-mask/-/plugin-mask-0.16.1.tgz", + "integrity": "sha512-snfiqHlVuj4bSFS0v96vo2PpqCDMe4JB+O++sMo5jF5mvGcGL6AIeLo8cYqPNpdO6BZpBJ8MY5El0Veckhr39Q==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-normalize": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-normalize/-/plugin-normalize-0.16.1.tgz", + "integrity": "sha512-dOQfIOvGLKDKXPU8xXWzaUeB0nvkosHw6Xg1WhS1Z5Q0PazByhaxOQkSKgUryNN/H+X7UdbDvlyh/yHf3ITRaw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-print": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-print/-/plugin-print-0.16.1.tgz", + "integrity": "sha512-ceWgYN40jbN4cWRxixym+csyVymvrryuKBQ+zoIvN5iE6OyS+2d7Mn4zlNgumSczb9GGyZZESIgVcBDA1ezq0Q==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "load-bmfont": "^1.4.0" + } + }, + "@jimp/plugin-resize": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-resize/-/plugin-resize-0.16.1.tgz", + "integrity": "sha512-u4JBLdRI7dargC04p2Ha24kofQBk3vhaf0q8FwSYgnCRwxfvh2RxvhJZk9H7Q91JZp6wgjz/SjvEAYjGCEgAwQ==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-rotate": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-rotate/-/plugin-rotate-0.16.1.tgz", + "integrity": "sha512-ZUU415gDQ0VjYutmVgAYYxC9Og9ixu2jAGMCU54mSMfuIlmohYfwARQmI7h4QB84M76c9hVLdONWjuo+rip/zg==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-scale": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-scale/-/plugin-scale-0.16.1.tgz", + "integrity": "sha512-jM2QlgThIDIc4rcyughD5O7sOYezxdafg/2Xtd1csfK3z6fba3asxDwthqPZAgitrLgiKBDp6XfzC07Y/CefUw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-shadow": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-shadow/-/plugin-shadow-0.16.1.tgz", + "integrity": "sha512-MeD2Is17oKzXLnsphAa1sDstTu6nxscugxAEk3ji0GV1FohCvpHBcec0nAq6/czg4WzqfDts+fcPfC79qWmqrA==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugin-threshold": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugin-threshold/-/plugin-threshold-0.16.1.tgz", + "integrity": "sha512-iGW8U/wiCSR0+6syrPioVGoSzQFt4Z91SsCRbgNKTAk7D+XQv6OI78jvvYg4o0c2FOlwGhqz147HZV5utoSLxA==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1" + } + }, + "@jimp/plugins": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/plugins/-/plugins-0.16.1.tgz", + "integrity": "sha512-c+lCqa25b+4q6mJZSetlxhMoYuiltyS+ValLzdwK/47+aYsq+kcJNl+TuxIEKf59yr9+5rkbpsPkZHLF/V7FFA==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/plugin-blit": "^0.16.1", + "@jimp/plugin-blur": "^0.16.1", + "@jimp/plugin-circle": "^0.16.1", + "@jimp/plugin-color": "^0.16.1", + "@jimp/plugin-contain": "^0.16.1", + "@jimp/plugin-cover": "^0.16.1", + "@jimp/plugin-crop": "^0.16.1", + "@jimp/plugin-displace": "^0.16.1", + "@jimp/plugin-dither": "^0.16.1", + "@jimp/plugin-fisheye": "^0.16.1", + "@jimp/plugin-flip": "^0.16.1", + "@jimp/plugin-gaussian": "^0.16.1", + "@jimp/plugin-invert": "^0.16.1", + "@jimp/plugin-mask": "^0.16.1", + "@jimp/plugin-normalize": "^0.16.1", + "@jimp/plugin-print": "^0.16.1", + "@jimp/plugin-resize": "^0.16.1", + "@jimp/plugin-rotate": "^0.16.1", + "@jimp/plugin-scale": "^0.16.1", + "@jimp/plugin-shadow": "^0.16.1", + "@jimp/plugin-threshold": "^0.16.1", + "timm": "^1.6.1" + } + }, + "@jimp/png": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/png/-/png-0.16.1.tgz", + "integrity": "sha512-iyWoCxEBTW0OUWWn6SveD4LePW89kO7ZOy5sCfYeDM/oTPLpR8iMIGvZpZUz1b8kvzFr27vPst4E5rJhGjwsdw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/utils": "^0.16.1", + "pngjs": "^3.3.3" + } + }, + "@jimp/tiff": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/tiff/-/tiff-0.16.1.tgz", + "integrity": "sha512-3K3+xpJS79RmSkAvFMgqY5dhSB+/sxhwTFA9f4AVHUK0oKW+u6r52Z1L0tMXHnpbAdR9EJ+xaAl2D4x19XShkQ==", + "requires": { + "@babel/runtime": "^7.7.2", + "utif": "^2.0.1" + } + }, + "@jimp/types": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/types/-/types-0.16.1.tgz", + "integrity": "sha512-g1w/+NfWqiVW4CaXSJyD28JQqZtm2eyKMWPhBBDCJN9nLCN12/Az0WFF3JUAktzdsEC2KRN2AqB1a2oMZBNgSQ==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/bmp": "^0.16.1", + "@jimp/gif": "^0.16.1", + "@jimp/jpeg": "^0.16.1", + "@jimp/png": "^0.16.1", + "@jimp/tiff": "^0.16.1", + "timm": "^1.6.1" + } + }, + "@jimp/utils": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/@jimp/utils/-/utils-0.16.1.tgz", + "integrity": "sha512-8fULQjB0x4LzUSiSYG6ZtQl355sZjxbv8r9PPAuYHzS9sGiSHJQavNqK/nKnpDsVkU88/vRGcE7t3nMU0dEnVw==", + "requires": { + "@babel/runtime": "^7.7.2", + "regenerator-runtime": "^0.13.3" + } + }, + "@jridgewell/gen-mapping": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.1.1.tgz", + "integrity": "sha512-sQXCasFk+U8lWYEe66WxRDOE9PjVz4vSM51fTu3Hw+ClTpUSQb718772vH3pyS5pShp6lvQM7SxgIDXXXmOX7w==", + "requires": { + "@jridgewell/set-array": "^1.0.0", + "@jridgewell/sourcemap-codec": "^1.4.10" + } + }, + "@jridgewell/resolve-uri": { + "version": "3.0.7", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.0.7.tgz", + "integrity": "sha512-8cXDaBBHOr2pQ7j77Y6Vp5VDT2sIqWyWQ56TjEq4ih/a4iST3dItRe8Q9fp0rrIl9DoKhWQtUQz/YpOxLkXbNA==" + }, + "@jridgewell/set-array": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.1.tgz", + "integrity": "sha512-Ct5MqZkLGEXTVmQYbGtx9SVqD2fqwvdubdps5D3djjAkgkKwT918VNOz65pEHFaYTeWcukmJmH5SwsA9Tn2ObQ==" + }, + "@jridgewell/sourcemap-codec": { + "version": "1.4.13", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.13.tgz", + "integrity": "sha512-GryiOJmNcWbovBxTfZSF71V/mXbgcV3MewDe3kIMCLyIh5e7SKAeUZs+rMnJ8jkMolZ/4/VsdBmMrw3l+VdZ3w==" + }, + "@jridgewell/trace-mapping": { + "version": "0.3.11", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.11.tgz", + "integrity": "sha512-RllI476aSMsxzeI9TtlSMoNTgHDxEmnl6GkkHwhr0vdL8W+0WuesyI8Vd3rBOfrwtPXbPxdT9ADJdiOKgzxPQA==", + "requires": { + "@jridgewell/resolve-uri": "^3.0.3", + "@jridgewell/sourcemap-codec": "^1.4.10" + } + }, + "@lezer/common": { + "version": "0.15.12", + "resolved": "https://registry.npmjs.org/@lezer/common/-/common-0.15.12.tgz", + "integrity": "sha512-edfwCxNLnzq5pBA/yaIhwJ3U3Kz8VAUOTRg0hhxaizaI1N+qxV7EXDv/kLCkLeq2RzSFvxexlaj5Mzfn2kY0Ig==" + }, + "@lezer/lr": { + "version": "0.15.8", + "resolved": "https://registry.npmjs.org/@lezer/lr/-/lr-0.15.8.tgz", + "integrity": "sha512-bM6oE6VQZ6hIFxDNKk8bKPa14hqFrV07J/vHGOeiAbJReIaQXmkVb6xQu4MR+JBTLa5arGRyAAjJe1qaQt3Uvg==", + "requires": { + "@lezer/common": "^0.15.0" + } + }, + "@mantine/core": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/core/-/core-4.2.5.tgz", + "integrity": "sha512-A/mixxpmCsA9acYVrCAurHrj/ikyLGksPqFILPJJpjC4uUDm85HHqVUa209VVLAKlvr26tX6CzDVqUeSe6s3Zg==", + "requires": { + "@mantine/styles": "4.2.5", + "@popperjs/core": "^2.9.3", + "@radix-ui/react-scroll-area": "^0.1.1", + "react-popper": "^2.2.5", + "react-textarea-autosize": "^8.3.2" + } + }, + "@mantine/hooks": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/hooks/-/hooks-4.2.5.tgz", + "integrity": "sha512-CEpdUXPAC28rXosgo/Wxvs3ch9qC+QYfqh4AFkOH0+EKdlXkD1xRN8vv6pd5AHJtRZvGf7CZDuGKSjWgssvgeA==", + "requires": {} + }, + "@mantine/ssr": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/ssr/-/ssr-4.2.5.tgz", + "integrity": "sha512-3Nt5PjCSKylAeyg1j6yct5hoX45J0jvxCwblphnhcYL+YVdNdaI2KL5Dg+KE/wcIAzOATquidUCFSHUwqcUVkg==", + "requires": { + "@emotion/cache": "11.7.1", + "@emotion/react": "11.7.1", + "@emotion/serialize": "1.0.2", + "@emotion/server": "11.4.0", + "@emotion/utils": "1.0.0", + "@mantine/styles": "4.2.5", + "csstype": "3.0.9", + "html-react-parser": "1.3.0" + }, + "dependencies": { + "csstype": { + "version": "3.0.9", + "resolved": "https://registry.npmmirror.com/csstype/-/csstype-3.0.9.tgz", + "integrity": "sha512-rpw6JPxK6Rfg1zLOYCSwle2GFOOsnjmDYDaBwEcwoOg4qlsIVCN789VkBZDJAGi4T07gI4YSutR43t9Zz4Lzuw==" + } + } + }, + "@mantine/styles": { + "version": "4.2.5", + "resolved": "https://registry.npmmirror.com/@mantine/styles/-/styles-4.2.5.tgz", + "integrity": "sha512-A6sIm3+Aa4ZqpaIqpmiaCmkAJI4ow9vwmCgpvuwWYCspBhWTWQWtdpnNyWfnTjszYY1uSnx9mb50JlIsglgLNQ==", + "requires": { + "@emotion/cache": "11.7.1", + "@emotion/react": "11.7.1", + "@emotion/serialize": "1.0.2", + "@emotion/utils": "1.0.0", + "clsx": "^1.1.1", + "csstype": "3.0.9" + }, + "dependencies": { + "csstype": { + "version": "3.0.9", + "resolved": "https://registry.npmmirror.com/csstype/-/csstype-3.0.9.tgz", + "integrity": "sha512-rpw6JPxK6Rfg1zLOYCSwle2GFOOsnjmDYDaBwEcwoOg4qlsIVCN789VkBZDJAGi4T07gI4YSutR43t9Zz4Lzuw==" + } + } + }, + "@microsoft/fetch-event-source": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/@microsoft/fetch-event-source/-/fetch-event-source-2.0.1.tgz", + "integrity": "sha512-W6CLUJ2eBMw3Rec70qrsEW0jOm/3twwJv21mrmj2yORiaVmVYGS4sSS5yUwvQc1ZlDLYGPnClVWmUUMagKNsfA==" + }, + "@mischnic/json-sourcemap": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/@mischnic/json-sourcemap/-/json-sourcemap-0.1.0.tgz", + "integrity": "sha512-dQb3QnfNqmQNYA4nFSN/uLaByIic58gOXq4Y4XqLOWmOrw73KmJPt/HLyG0wvn1bnR6mBKs/Uwvkh+Hns1T0XA==", + "requires": { + "@lezer/common": "^0.15.7", + "@lezer/lr": "^0.15.4", + "json5": "^2.2.1" + } + }, + "@nodelib/fs.scandir": { + "version": "2.1.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", + "integrity": "sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g==", + "requires": { + "@nodelib/fs.stat": "2.0.5", + "run-parallel": "^1.1.9" + } + }, + "@nodelib/fs.stat": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz", + "integrity": "sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A==" + }, + "@nodelib/fs.walk": { + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz", + "integrity": "sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg==", + "requires": { + "@nodelib/fs.scandir": "2.1.5", + "fastq": "^1.6.0" + } + }, + "@parcel/bundler-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/bundler-default/-/bundler-default-2.5.0.tgz", + "integrity": "sha512-7CJzE17SirCXjcRgBcnqWO/5EOA1raq/3OIKtT4cxbjpDQGHZpjpEEZiMNRpEpdNMxDSlsG8mAkXTYGL2VVWRw==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/cache": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/cache/-/cache-2.5.0.tgz", + "integrity": "sha512-3kOO3cZQv0FAKhrMHGLdb4Qtzpmy78Q6jPN3u8eCY4yqeDTnyQBZvWNHoyCm5WlmL8y6Q6REYMbETLxSH1ggAQ==", + "requires": { + "@parcel/fs": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/utils": "2.5.0", + "lmdb": "2.2.4" + }, + "dependencies": { + "lmdb": { + "version": "2.2.4", + "resolved": "https://registry.npmjs.org/lmdb/-/lmdb-2.2.4.tgz", + "integrity": "sha512-gto+BB2uEob8qRiTlOq+R3uX0YNHsX9mjxj9Sbdue/LIKqu6IlZjrsjKeGyOMquc/474GEqFyX2pdytpydp0rQ==", + "requires": { + "msgpackr": "^1.5.4", + "nan": "^2.14.2", + "node-gyp-build": "^4.2.3", + "ordered-binary": "^1.2.4", + "weak-lru-cache": "^1.2.2" + } + } + } + }, + "@parcel/codeframe": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/codeframe/-/codeframe-2.5.0.tgz", + "integrity": "sha512-qafqL8Vu2kr932cCWESoDEEoAeKVi7/xdzTBuhzEJng1AfmRT0rCbt/P4ao3RjiDyozPSjXsHOqM6GDZcto4eQ==", + "requires": { + "chalk": "^4.1.0" + } + }, + "@parcel/compressor-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/compressor-raw/-/compressor-raw-2.5.0.tgz", + "integrity": "sha512-I5Zs+2f1ue4sTPdfT8BNsLfTZl48sMWLk2Io3elUJjH/SS9kO7ut5ChkuJtt77ZS35m0OF+ZCt3ICTJdnDG8eA==", + "requires": { + "@parcel/plugin": "2.5.0" + } + }, + "@parcel/core": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/core/-/core-2.5.0.tgz", + "integrity": "sha512-dygDmPsfAYJKTnUftcbEzjCik7AAaPbFvJW8ETYz8diyjkAG9y6hvCAZIrJE5pNOjFzg32en4v4UWv8Sqlzl9g==", + "requires": { + "@mischnic/json-sourcemap": "^0.1.0", + "@parcel/cache": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/events": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/graph": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/package-manager": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "abortcontroller-polyfill": "^1.1.9", + "base-x": "^3.0.8", + "browserslist": "^4.6.6", + "clone": "^2.1.1", + "dotenv": "^7.0.0", + "dotenv-expand": "^5.1.0", + "json5": "^2.2.0", + "msgpackr": "^1.5.4", + "nullthrows": "^1.1.1", + "semver": "^5.7.1" + }, + "dependencies": { + "dotenv": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-7.0.0.tgz", + "integrity": "sha512-M3NhsLbV1i6HuGzBUH8vXrtxOk+tWmzWKDMbAVSUp3Zsjm7ywFeuwrUXhmhQyRK1q5B5GGy7hcXPbj3bnfZg2g==" + }, + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + } + } + }, + "@parcel/diagnostic": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/diagnostic/-/diagnostic-2.5.0.tgz", + "integrity": "sha512-KiMGGRpEV7wl5gjcxBKcgX84a+cG+IEn94gwy5LK3lENR09nuKShqqgKGAmj/17CobJgw1QNP94/H4Md+oxIWg==", + "requires": { + "@mischnic/json-sourcemap": "^0.1.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/events": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/events/-/events-2.5.0.tgz", + "integrity": "sha512-Gc2LPwL1H34Ony5MENbKZg7wvCscZ4x9y7Fu92sfbdWpLo3K13hVtsX3TMIIgYt3B7R7OmO8yR880U2T+JfVkQ==" + }, + "@parcel/fs": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/fs/-/fs-2.5.0.tgz", + "integrity": "sha512-YYr14BWtx/bJ+hu6PPQQ6G/3omOTWgVqEw+UFI3iQH3P6+e0LRXW/Ja1yAcJeepGcTwIP0opnXZBQOm8PBQ2SA==", + "requires": { + "@parcel/fs-search": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/watcher": "^2.0.0", + "@parcel/workers": "2.5.0" + } + }, + "@parcel/fs-search": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/fs-search/-/fs-search-2.5.0.tgz", + "integrity": "sha512-uBONkz9ZCNSOqbPGWJY3MNl+pqBTfvzHH9+4UhzHEHPArvK2oD0+syYPVE60+zGrxybXTESYMCJp4bHvH6Z2hA==", + "requires": { + "detect-libc": "^1.0.3" + } + }, + "@parcel/graph": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/graph/-/graph-2.5.0.tgz", + "integrity": "sha512-qa2VtG08dJyTaWrxYAkMIlkoDRSPoiqLDNxxHKplkcxAjXBUw0/AkWaz82VO5r1G6jfOj+nM30ajH9uygZYwbw==", + "requires": { + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/hash": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/hash/-/hash-2.5.0.tgz", + "integrity": "sha512-47JL0XpB7UvIW6Ijf8vv+yVMt9dLvB/lRlBHFmAkmovisueVMVbYD7smxVZnCSehD8UH8BcymKbMzyL5dimgoQ==", + "requires": { + "detect-libc": "^1.0.3", + "xxhash-wasm": "^0.4.2" + } + }, + "@parcel/logger": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/logger/-/logger-2.5.0.tgz", + "integrity": "sha512-pT1L3ceH6trL1N3I3r2HawPjz/PCubOo/Kazu7IeXsMsKVjj1a6AeieZHzkNZIbhiGPtm/cHbBNLz2zTWDLeOA==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/events": "2.5.0" + } + }, + "@parcel/markdown-ansi": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/markdown-ansi/-/markdown-ansi-2.5.0.tgz", + "integrity": "sha512-ixkNF3KWIqxMlfxTe9Gb2cp/uNmklQev8VEUxujMVxmUfGyQs4859zdJIQlIinabWYhArhsXATkVf3MzCUN6TQ==", + "requires": { + "chalk": "^4.1.0" + } + }, + "@parcel/namer-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/namer-default/-/namer-default-2.5.0.tgz", + "integrity": "sha512-ahGQqHJzsWE5Qux8zXMAU+lyNBOl+ZpcOFzRGE2DWOsmAlytsHl7DBVCQvzUyNBFg1/HmIj+7D4efv2kjR7rTg==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/node-resolver-core": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/node-resolver-core/-/node-resolver-core-2.5.0.tgz", + "integrity": "sha512-XQvpguiIwQcu75cscLDFOVhjsjuPzXbuMaaZ7XxxUEl0PscIgu/GfKYxTfTruN3cRl+CaQH6qBAMfjLaFng6lQ==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/optimizer-terser": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/optimizer-terser/-/optimizer-terser-2.5.0.tgz", + "integrity": "sha512-PZ3UHBGfjE49/Jloopsd38Hxg4qzsrdepWP53mCuVP7Aw605Y4QtYuB1ho3VV0oXfKQVq+uI7lVIBsuW4K6vqA==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1", + "terser": "^5.2.0" + } + }, + "@parcel/package-manager": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/package-manager/-/package-manager-2.5.0.tgz", + "integrity": "sha512-zTuF55/lITUjw9dUU/X0HiF++589xbPXw/zUiG9T6s8BQThLvrxAhYP89S719pw7cTqDimGkTxnIuK+a0djEkg==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "semver": "^5.7.1" + }, + "dependencies": { + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + } + } + }, + "@parcel/packager-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/packager-js/-/packager-js-2.5.0.tgz", + "integrity": "sha512-aJAKOTgXdxO3V9O7+2DCVOtne128WwXmUAOVThnMRo7f3zMVSAR7Mxc9pEsuTzPfj8UBXgFBRfdJUSCgsMxiSw==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "globals": "^13.2.0", + "nullthrows": "^1.1.1" + }, + "dependencies": { + "globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "requires": { + "type-fest": "^0.20.2" + } + } + } + }, + "@parcel/packager-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/packager-raw/-/packager-raw-2.5.0.tgz", + "integrity": "sha512-aHV0oogeiqxhxS1lsttw15EvG3DDWK3FV7+F+7hoaAy+xg89K56NTp6j43Jtw9iyU1/HnZRGBE2hF3C7N73oKw==", + "requires": { + "@parcel/plugin": "2.5.0" + } + }, + "@parcel/plugin": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/plugin/-/plugin-2.5.0.tgz", + "integrity": "sha512-obtb6/Gql6YFQ86bdv75A2Noabx8679reFZeyfKKf0L7Lppx4DFQetXwM9XVy7Gx6hJ1Ekm3UMuuIyVJk33YHQ==", + "requires": { + "@parcel/types": "2.5.0" + } + }, + "@parcel/reporter-dev-server": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/reporter-dev-server/-/reporter-dev-server-2.5.0.tgz", + "integrity": "sha512-wvxAiW42AxJ3B8jtvowJcP4/cTV8zY48SfKg61YKYu1yUO+TtyJIjHQzDW2XuT34cIGFY97Gr0i+AVu44RyUuQ==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0" + } + }, + "@parcel/resolver-default": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/resolver-default/-/resolver-default-2.5.0.tgz", + "integrity": "sha512-39PkZpVr/+iYS11u+lA84vIsKm/yisltTVmUjlYsDnExiuV1c8OSbSdYZ3JMx+7CYPE0bWbosX2AGilIwIMWpQ==", + "requires": { + "@parcel/node-resolver-core": "2.5.0", + "@parcel/plugin": "2.5.0" + } + }, + "@parcel/runtime-browser-hmr": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-browser-hmr/-/runtime-browser-hmr-2.5.0.tgz", + "integrity": "sha512-oPAo8Zf06gXCpt41nyvK7kv2HH1RrHAGgOqttyjStwAFlm5MZKs7BgtJzO58LfJN8g3sMY0cNdG17fB/4f8q6Q==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0" + } + }, + "@parcel/runtime-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-js/-/runtime-js-2.5.0.tgz", + "integrity": "sha512-gPC2PbNAiooULP71wF5twe4raekuXsR1Hw/ahITDoqsZdXHzG3CkoCjYL3CkmBGiKQgMMocCyN1E2oBzAH8Kyw==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/runtime-react-refresh": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-react-refresh/-/runtime-react-refresh-2.5.0.tgz", + "integrity": "sha512-+8RuDKFdFYIQTrXG4MRhG9XqkkYEHn0zxKyOJ/IkDDfSEhY0na+EyhrneFUwIvDX63gLPkxceXAg0gwBqXPK/Q==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "react-refresh": "^0.9.0" + } + }, + "@parcel/runtime-service-worker": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/runtime-service-worker/-/runtime-service-worker-2.5.0.tgz", + "integrity": "sha512-STuDlU0fPXeWpAmbayY7o04F0eHy6FTOFeT5KQ0PTxtdEa3Ey8QInP/NVE52Yv0aVQtesWukGrNEFCERlkbFRw==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "nullthrows": "^1.1.1" + } + }, + "@parcel/source-map": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/@parcel/source-map/-/source-map-2.0.2.tgz", + "integrity": "sha512-NnUrPYLpYB6qyx2v6bcRPn/gVigmGG6M6xL8wIg/i0dP1GLkuY1nf+Hqdf63FzPTqqT7K3k6eE5yHPQVMO5jcA==", + "requires": { + "detect-libc": "^1.0.3" + } + }, + "@parcel/transformer-js": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-js/-/transformer-js-2.5.0.tgz", + "integrity": "sha512-Cp8Ic+Au3OcskCRZszmo47z3bqcZ7rfPv2xZYXpXY2TzEc3IV0bKje57bZektoY8LW9LkYM9iBO/WhkVoT6LIg==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/plugin": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/utils": "2.5.0", + "@parcel/workers": "2.5.0", + "@swc/helpers": "^0.3.6", + "browserslist": "^4.6.6", + "detect-libc": "^1.0.3", + "nullthrows": "^1.1.1", + "regenerator-runtime": "^0.13.7", + "semver": "^5.7.1" + }, + "dependencies": { + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + } + } + }, + "@parcel/transformer-json": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-json/-/transformer-json-2.5.0.tgz", + "integrity": "sha512-661sByA7TkR6Lmxt+hqV4h2SAt+7lgc58DzmUYArpEl1fQnMuQuaB0kQeHzi6fDD2+2G6o7EC+DuwBZKa479TA==", + "requires": { + "@parcel/plugin": "2.5.0", + "json5": "^2.2.0" + } + }, + "@parcel/transformer-raw": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-raw/-/transformer-raw-2.5.0.tgz", + "integrity": "sha512-I3zjE1u9+Wj90Qqs1V2FTm6iC6SAyOVUthwVZkZey+qbQG/ok682Ez2XjLu7MyQCo9BJNwF/nfOa1hHr3MaJEQ==", + "requires": { + "@parcel/plugin": "2.5.0" + } + }, + "@parcel/transformer-react-refresh-wrap": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/transformer-react-refresh-wrap/-/transformer-react-refresh-wrap-2.5.0.tgz", + "integrity": "sha512-VPqVBxhTN4OQwcjsdyxrv+smjAm4s6dbSWAplgPwdOITMv+a0tjhhJU37WnRC+xxTrbEqRcOt96JvGOkPb8i7g==", + "requires": { + "@parcel/plugin": "2.5.0", + "@parcel/utils": "2.5.0", + "react-refresh": "^0.9.0" + } + }, + "@parcel/types": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/types/-/types-2.5.0.tgz", + "integrity": "sha512-bA0fhG6aXSGYEVo5Dt96x6lseUQHeVZVzgmiRdZsvb614Gvx22ItfaKhPmAVbM9vzbObZDHl9l9G2Ovw8Xve4g==", + "requires": { + "@parcel/cache": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/fs": "2.5.0", + "@parcel/package-manager": "2.5.0", + "@parcel/source-map": "^2.0.0", + "@parcel/workers": "2.5.0", + "utility-types": "^3.10.0" + } + }, + "@parcel/utils": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/utils/-/utils-2.5.0.tgz", + "integrity": "sha512-kaLGXtQuOOH55KZqXdYDvczhh3mk2eeTVqrrXuuihGjbLKYFlUW2tFDm+5r2s9nCPwTQxOO43ZEOCKSnia+e4w==", + "requires": { + "@parcel/codeframe": "2.5.0", + "@parcel/diagnostic": "2.5.0", + "@parcel/hash": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/markdown-ansi": "2.5.0", + "@parcel/source-map": "^2.0.0", + "chalk": "^4.1.0" + } + }, + "@parcel/watcher": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@parcel/watcher/-/watcher-2.0.5.tgz", + "integrity": "sha512-x0hUbjv891omnkcHD7ZOhiyyUqUUR6MNjq89JhEI3BxppeKWAm6NPQsqqRrAkCJBogdT/o/My21sXtTI9rJIsw==", + "requires": { + "node-addon-api": "^3.2.1", + "node-gyp-build": "^4.3.0" + } + }, + "@parcel/workers": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@parcel/workers/-/workers-2.5.0.tgz", + "integrity": "sha512-/Ow5OKJWs+9OzV3Jy4J++VnbNx0j3ls/M1CGVBLiBWyCada9DMtquYoBQ4Sk6Uam50BKkIFYetGOeXPNQyyMjg==", + "requires": { + "@parcel/diagnostic": "2.5.0", + "@parcel/logger": "2.5.0", + "@parcel/types": "2.5.0", + "@parcel/utils": "2.5.0", + "chrome-trace-event": "^1.0.2", + "nullthrows": "^1.1.1" + } + }, + "@pmmmwh/react-refresh-webpack-plugin": { + "version": "0.4.3", + "resolved": "https://registry.npmjs.org/@pmmmwh/react-refresh-webpack-plugin/-/react-refresh-webpack-plugin-0.4.3.tgz", + "integrity": "sha512-br5Qwvh8D2OQqSXpd1g/xqXKnK0r+Jz6qVKBbWmpUcrbGOxUrf39V5oZ1876084CGn18uMdR5uvPqBv9UqtBjQ==", + "requires": { + "ansi-html": "^0.0.7", + "error-stack-parser": "^2.0.6", + "html-entities": "^1.2.1", + "native-url": "^0.2.6", + "schema-utils": "^2.6.5", + "source-map": "^0.7.3" + }, + "dependencies": { + "ansi-html": { + "version": "0.0.7", + "resolved": "https://registry.npmjs.org/ansi-html/-/ansi-html-0.0.7.tgz", + "integrity": "sha1-gTWEAhliqenm/QOflA0S9WynhZ4=" + }, + "html-entities": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/html-entities/-/html-entities-1.4.0.tgz", + "integrity": "sha512-8nxjcBcd8wovbeKx7h3wTji4e6+rhaVuPNpMqwWgnHh+N9ToqsCs6XztWRBPQ+UtzsoMAdKZtUENoVzU/EMtZA==" + } + } + }, + "@popperjs/core": { + "version": "2.11.5", + "resolved": "https://registry.npmmirror.com/@popperjs/core/-/core-2.11.5.tgz", + "integrity": "sha512-9X2obfABZuDVLCgPK9aX0a/x4jaOEweTTWE2+9sr0Qqqevj2Uv5XorvusThmc9XGYpS9yI+fhh8RTafBtGposw==" + }, + "@radix-ui/number": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/number/-/number-0.1.0.tgz", + "integrity": "sha512-rpf6QiOWLHAkM4FEMYu9i+5Jr8cKT893+R4mPpcdsy4LD7omr9JfdOqj/h/xPA5+EcVrpMMlU6rrRYpUB5UI8g==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/primitive": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/primitive/-/primitive-0.1.0.tgz", + "integrity": "sha512-tqxZKybwN5Fa3VzZry4G6mXAAb9aAqKmPtnVbZpL0vsBwvOHTBwsjHVPXylocYLwEtBY9SCe665bYnNB515uoA==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/react-compose-refs": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-compose-refs/-/react-compose-refs-0.1.0.tgz", + "integrity": "sha512-eyclbh+b77k+69Dk72q3694OHrn9B3QsoIRx7ywX341U9RK1ThgQjMFZoPtmZNQTksXHLNEiefR8hGVeFyInGg==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/react-context": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-context/-/react-context-0.1.1.tgz", + "integrity": "sha512-PkyVX1JsLBioeu0jB9WvRpDBBLtLZohVDT3BB5CTSJqActma8S8030P57mWZb4baZifMvN7KKWPAA40UmWKkQg==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/react-presence": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-presence/-/react-presence-0.1.2.tgz", + "integrity": "sha512-3BRlFZraooIUfRlyN+b/Xs5hq1lanOOo/+3h6Pwu2GMFjkGKKa4Rd51fcqGqnVlbr3jYg+WLuGyAV4KlgqwrQw==", + "requires": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-compose-refs": "0.1.0", + "@radix-ui/react-use-layout-effect": "0.1.0" + } + }, + "@radix-ui/react-primitive": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-primitive/-/react-primitive-0.1.4.tgz", + "integrity": "sha512-6gSl2IidySupIMJFjYnDIkIWRyQdbu/AHK7rbICPani+LW4b0XdxBXc46og/iZvuwW8pjCS8I2SadIerv84xYA==", + "requires": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-slot": "0.1.2" + } + }, + "@radix-ui/react-scroll-area": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-scroll-area/-/react-scroll-area-0.1.4.tgz", + "integrity": "sha512-QHxRsjy+hsHwQYJ9cCNgSJ5+6ioZu1KhwD1UOXoHNciuFGMX08v+uJPKXIz+ySv03Rx6cOz6f/Fk5aPHRMFi/A==", + "requires": { + "@babel/runtime": "^7.13.10", + "@radix-ui/number": "0.1.0", + "@radix-ui/primitive": "0.1.0", + "@radix-ui/react-compose-refs": "0.1.0", + "@radix-ui/react-context": "0.1.1", + "@radix-ui/react-presence": "0.1.2", + "@radix-ui/react-primitive": "0.1.4", + "@radix-ui/react-use-callback-ref": "0.1.0", + "@radix-ui/react-use-direction": "0.1.0", + "@radix-ui/react-use-layout-effect": "0.1.0" + } + }, + "@radix-ui/react-slot": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-slot/-/react-slot-0.1.2.tgz", + "integrity": "sha512-ADkqfL+agEzEguU3yS26jfB50hRrwf7U4VTwAOZEmi/g+ITcBWe12yM46ueS/UCIMI9Py+gFUaAdxgxafFvY2Q==", + "requires": { + "@babel/runtime": "^7.13.10", + "@radix-ui/react-compose-refs": "0.1.0" + } + }, + "@radix-ui/react-use-callback-ref": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-0.1.0.tgz", + "integrity": "sha512-Va041McOFFl+aV+sejvl0BS2aeHx86ND9X/rVFmEFQKTXCp6xgUK0NGUAGcgBlIjnJSbMYPGEk1xKSSlVcN2Aw==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/react-use-direction": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-direction/-/react-use-direction-0.1.0.tgz", + "integrity": "sha512-NajpY/An9TCPSfOVkgWIdXJV+VuWl67PxB6kOKYmtNAFHvObzIoh8o0n9sAuwSAyFCZVq211FEf9gvVDRhOyiA==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@radix-ui/react-use-layout-effect": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-0.1.0.tgz", + "integrity": "sha512-+wdeS51Y+E1q1Wmd+1xSSbesZkpVj4jsg0BojCbopWvgq5iBvixw5vgemscdh58ep98BwUbsFYnrywFhV9yrVg==", + "requires": { + "@babel/runtime": "^7.13.10" + } + }, + "@sideway/address": { + "version": "4.1.4", + "resolved": "https://registry.npmjs.org/@sideway/address/-/address-4.1.4.tgz", + "integrity": "sha512-7vwq+rOHVWjyXxVlR76Agnvhy8I9rpzjosTESvmhNeXOXdZZB15Fl+TI9x1SiHZH5Jv2wTGduSxFDIaq0m3DUw==", + "requires": { + "@hapi/hoek": "^9.0.0" + } + }, + "@sideway/formula": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@sideway/formula/-/formula-3.0.0.tgz", + "integrity": "sha512-vHe7wZ4NOXVfkoRb8T5otiENVlT7a3IAiw7H5M2+GO+9CDgcVUUsX1zalAztCmwyOr2RUTGJdgB+ZvSVqmdHmg==" + }, + "@sideway/pinpoint": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@sideway/pinpoint/-/pinpoint-2.0.0.tgz", + "integrity": "sha512-RNiOoTPkptFtSVzQevY/yWtZwf/RxyVnPy/OcA9HBM3MlGDnBEYL5B41H0MTn0Uec8Hi+2qUtTfG2WWZBmMejQ==" + }, + "@sindresorhus/is": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-4.6.0.tgz", + "integrity": "sha512-t09vSN3MdfsyCHoFcTRCH/iUtG7OJ0CsjzB8cjAmKc/va/kIgeDI/TxsigdncE/4be734m0cvIYwNaV4i2XqAw==" + }, + "@sindresorhus/slugify": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@sindresorhus/slugify/-/slugify-1.1.2.tgz", + "integrity": "sha512-V9nR/W0Xd9TSGXpZ4iFUcFGhuOJtZX82Fzxj1YISlbSgKvIiNa7eLEZrT0vAraPOt++KHauIVNYgGRgjc13dXA==", + "requires": { + "@sindresorhus/transliterate": "^0.1.1", + "escape-string-regexp": "^4.0.0" + }, + "dependencies": { + "escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==" + } + } + }, + "@sindresorhus/transliterate": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/@sindresorhus/transliterate/-/transliterate-0.1.2.tgz", + "integrity": "sha512-5/kmIOY9FF32nicXH+5yLNTX4NJ4atl7jRgqAJuIn/iyDFXBktOKDxCvyGE/EzmF4ngSUvjXxQUQlQiZ5lfw+w==", + "requires": { + "escape-string-regexp": "^2.0.0", + "lodash.deburr": "^4.1.0" + }, + "dependencies": { + "escape-string-regexp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", + "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==" + } + } + }, + "@swc/helpers": { + "version": "0.3.13", + "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.3.13.tgz", + "integrity": "sha512-A1wswJhnqaLRn8uYVQ8YiNTtY5i/JIPmV08EXXjjTresIkUVUEUaFv/wXVhGXfRNYMvHPkuoMR1Nb6NgpxGjNg==", + "requires": { + "tslib": "^2.4.0" + }, + "dependencies": { + "tslib": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.4.0.tgz", + "integrity": "sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==" + } + } + }, + "@szmarczak/http-timer": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-4.0.6.tgz", + "integrity": "sha512-4BAffykYOgO+5nzBWYwE3W90sBgLJoUPRWWcL8wlyiM8IB8ipJz3UMJ9KXQd1RKQXpKp8Tutn80HZtWsu2u76w==", + "requires": { + "defer-to-connect": "^2.0.0" + } + }, + "@tokenizer/token": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/@tokenizer/token/-/token-0.3.0.tgz", + "integrity": "sha512-OvjF+z51L3ov0OyAU0duzsYuvO01PH7x4t6DJx+guahgTnBHkhJdG7soQeTSFLWN3efnHyibZ4Z8l2EuWwJN3A==" + }, + "@trysound/sax": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/@trysound/sax/-/sax-0.2.0.tgz", + "integrity": "sha512-L7z9BgrNEcYyUYtF+HaEfiS5ebkh9jXqbszz7pC0hRBPaatV0XjSD3+eHrpqFemQfgwiFF0QPIarnIihIDn7OA==" + }, + "@turist/fetch": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@turist/fetch/-/fetch-7.2.0.tgz", + "integrity": "sha512-2x7EGw+6OJ29phunsbGvtxlNmSfcuPcyYudkMbi8gARCP9eJ1CtuMvnVUHL//O9Ixi9SJiug8wNt6lj86pN8XQ==", + "requires": { + "@types/node-fetch": "2" + } + }, + "@turist/time": { + "version": "0.0.2", + "resolved": "https://registry.npmjs.org/@turist/time/-/time-0.0.2.tgz", + "integrity": "sha512-qLOvfmlG2vCVw5fo/oz8WAZYlpe5a5OurgTj3diIxJCdjRHpapC+vQCz3er9LV79Vcat+DifBjeAhOAdmndtDQ==" + }, + "@types/cacheable-request": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/@types/cacheable-request/-/cacheable-request-6.0.2.tgz", + "integrity": "sha512-B3xVo+dlKM6nnKTcmm5ZtY/OL8bOAOd2Olee9M1zft65ox50OzjEHW91sDiU9j6cvW8Ejg1/Qkf4xd2kugApUA==", + "requires": { + "@types/http-cache-semantics": "*", + "@types/keyv": "*", + "@types/node": "*", + "@types/responselike": "*" + } + }, + "@types/common-tags": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/@types/common-tags/-/common-tags-1.8.1.tgz", + "integrity": "sha512-20R/mDpKSPWdJs5TOpz3e7zqbeCNuMCPhV7Yndk9KU2Rbij2r5W4RzwDPkzC+2lzUqXYu9rFzTktCBnDjHuNQg==" + }, + "@types/component-emitter": { + "version": "1.2.11", + "resolved": "https://registry.npmjs.org/@types/component-emitter/-/component-emitter-1.2.11.tgz", + "integrity": "sha512-SRXjM+tfsSlA9VuG8hGO2nft2p8zjXCK1VcC6N4NXbBbYbSia9kzCChYQajIjzIqOOOuh5Ock6MmV2oux4jDZQ==" + }, + "@types/configstore": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/@types/configstore/-/configstore-2.1.1.tgz", + "integrity": "sha1-zR6FU2M60xhcPy8jns/10mQ+krY=" + }, + "@types/cookie": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/@types/cookie/-/cookie-0.4.1.tgz", + "integrity": "sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q==" + }, + "@types/cors": { + "version": "2.8.12", + "resolved": "https://registry.npmjs.org/@types/cors/-/cors-2.8.12.tgz", + "integrity": "sha512-vt+kDhq/M2ayberEtJcIN/hxXy1Pk+59g2FV/ZQceeaTyCtCucjL2Q7FXlFjtWn4n15KCr1NE2lNNFhp0lEThw==" + }, + "@types/debug": { + "version": "0.0.30", + "resolved": "https://registry.npmjs.org/@types/debug/-/debug-0.0.30.tgz", + "integrity": "sha512-orGL5LXERPYsLov6CWs3Fh6203+dXzJkR7OnddIr2514Hsecwc8xRpzCapshBbKFImCsvS/mk6+FWiN5LyZJAQ==" + }, + "@types/eslint": { + "version": "7.29.0", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-7.29.0.tgz", + "integrity": "sha512-VNcvioYDH8/FxaeTKkM4/TiTwt6pBV9E3OfGmvaw8tPl0rrHCJ4Ll15HRT+pMiFAf/MLQvAzC+6RzUMEL9Ceng==", + "requires": { + "@types/estree": "*", + "@types/json-schema": "*" + } + }, + "@types/eslint-scope": { + "version": "3.7.3", + "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.3.tgz", + "integrity": "sha512-PB3ldyrcnAicT35TWPs5IcwKD8S333HMaa2VVv4+wdvebJkjWuW/xESoB8IwRcog8HYVYamb1g/R31Qv5Bx03g==", + "requires": { + "@types/eslint": "*", + "@types/estree": "*" + } + }, + "@types/estree": { + "version": "0.0.51", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-0.0.51.tgz", + "integrity": "sha512-CuPgU6f3eT/XgKKPqKd/gLZV1Xmvf1a2R5POBOGQa6uv82xpls89HU5zKeVoyR8XzHd1RGNOlQlvUe3CFkjWNQ==" + }, + "@types/get-port": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/@types/get-port/-/get-port-3.2.0.tgz", + "integrity": "sha512-TiNg8R1kjDde5Pub9F9vCwZA/BNW9HeXP5b9j7Qucqncy/McfPZ6xze/EyBdXS5FhMIGN6Fx3vg75l5KHy3V1Q==" + }, + "@types/glob": { + "version": "5.0.37", + "resolved": "https://registry.npmjs.org/@types/glob/-/glob-5.0.37.tgz", + "integrity": "sha512-ATA/xrS7CZ3A2WCPVY4eKdNpybq56zqlTirnHhhyOztZM/lPxJzusOBI3BsaXbu6FrUluqzvMlI4sZ6BDYMlMg==", + "requires": { + "@types/minimatch": "*", + "@types/node": "*" + } + }, + "@types/hast": { + "version": "2.3.4", + "resolved": "https://registry.npmmirror.com/@types/hast/-/hast-2.3.4.tgz", + "integrity": "sha512-wLEm0QvaoawEDoTRwzTXp4b4jpwiJDvR5KMnFnVodm3scufTlBOWRD6N1OBf9TZMhjlNsSfcO5V+7AF4+Vy+9g==", + "requires": { + "@types/unist": "*" + } + }, + "@types/http-cache-semantics": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/@types/http-cache-semantics/-/http-cache-semantics-4.0.1.tgz", + "integrity": "sha512-SZs7ekbP8CN0txVG2xVRH6EgKmEm31BOxA07vkFaETzZz1xh+cbt8BcI0slpymvwhx5dlFnQG2rTlPVQn+iRPQ==" + }, + "@types/http-proxy": { + "version": "1.17.9", + "resolved": "https://registry.npmjs.org/@types/http-proxy/-/http-proxy-1.17.9.tgz", + "integrity": "sha512-QsbSjA/fSk7xB+UXlCT3wHBy5ai9wOcNDWwZAtud+jXhwOM3l+EYZh8Lng4+/6n8uar0J7xILzqftJdJ/Wdfkw==", + "requires": { + "@types/node": "*" + } + }, + "@types/json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@types/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha512-3YP80IxxFJB4b5tYC2SUPwkg0XQLiu0nWvhRgEatgjf+29IcWO9X1k8xRv5DGssJ/lCrjYTjQPcobJr2yWIVuQ==" + }, + "@types/json-schema": { + "version": "7.0.11", + "resolved": "https://registry.npmjs.org/@types/json-schema/-/json-schema-7.0.11.tgz", + "integrity": "sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ==" + }, + "@types/json5": { + "version": "0.0.29", + "resolved": "https://registry.npmjs.org/@types/json5/-/json5-0.0.29.tgz", + "integrity": "sha1-7ihweulOEdK4J7y+UnC86n8+ce4=" + }, + "@types/keyv": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@types/keyv/-/keyv-3.1.4.tgz", + "integrity": "sha512-BQ5aZNSCpj7D6K2ksrRCTmKRLEpnPvWDiLPfoGyhZ++8YtiK9d/3DBKPJgry359X/P1PfruyYwvnvwFjuEiEIg==", + "requires": { + "@types/node": "*" + } + }, + "@types/lodash": { + "version": "4.14.182", + "resolved": "https://registry.npmjs.org/@types/lodash/-/lodash-4.14.182.tgz", + "integrity": "sha512-/THyiqyQAP9AfARo4pF+aCGcyiQ94tX/Is2I7HofNRqoYLgN1PBoOWu2/zTA5zMxzP5EFutMtWtGAFRKUe961Q==" + }, + "@types/mdast": { + "version": "3.0.10", + "resolved": "https://registry.npmmirror.com/@types/mdast/-/mdast-3.0.10.tgz", + "integrity": "sha512-W864tg/Osz1+9f4lrGTZpCSO5/z4608eUp19tbozkq2HJK6i3z1kT0H9tlADXuYIb1YYOBByU4Jsqkk75q48qA==", + "requires": { + "@types/unist": "*" + } + }, + "@types/minimatch": { + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/@types/minimatch/-/minimatch-3.0.5.tgz", + "integrity": "sha512-Klz949h02Gz2uZCMGwDUSDS1YBlTdDDgbWHi+81l29tQALUtvz4rAYi5uoVhE5Lagoq6DeqAUlbrHvW/mXDgdQ==" + }, + "@types/minimist": { + "version": "1.2.2", + "resolved": "https://registry.npmmirror.com/@types/minimist/-/minimist-1.2.2.tgz", + "integrity": "sha512-jhuKLIRrhvCPLqwPcx6INqmKeiA5EWrsCOPhrlFSrbrmU4ZMPjj5Ul/oLCMDO98XRUIwVm78xICz4EPCektzeQ==" + }, + "@types/mkdirp": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/@types/mkdirp/-/mkdirp-0.5.2.tgz", + "integrity": "sha512-U5icWpv7YnZYGsN4/cmh3WD2onMY0aJIiTE6+51TwJCttdHvtCYmkBNOobHlXwrJRL0nkH9jH4kD+1FAdMN4Tg==", + "requires": { + "@types/node": "*" + } + }, + "@types/node": { + "version": "17.0.32", + "resolved": "https://registry.npmjs.org/@types/node/-/node-17.0.32.tgz", + "integrity": "sha512-eAIcfAvhf/BkHcf4pkLJ7ECpBAhh9kcxRBpip9cTiO+hf+aJrsxYxBeS6OXvOd9WqNAJmavXVpZvY1rBjNsXmw==" + }, + "@types/node-fetch": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/@types/node-fetch/-/node-fetch-2.6.1.tgz", + "integrity": "sha512-oMqjURCaxoSIsHSr1E47QHzbmzNR5rK8McHuNb11BOM9cHcIK3Avy0s/b2JlXHoQGTYS3NsvWzV1M0iK7l0wbA==", + "requires": { + "@types/node": "*", + "form-data": "^3.0.0" + }, + "dependencies": { + "form-data": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-3.0.1.tgz", + "integrity": "sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg==", + "requires": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + } + } + } + }, + "@types/normalize-package-data": { + "version": "2.4.1", + "resolved": "https://registry.npmmirror.com/@types/normalize-package-data/-/normalize-package-data-2.4.1.tgz", + "integrity": "sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw==" + }, + "@types/parse-json": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/@types/parse-json/-/parse-json-4.0.0.tgz", + "integrity": "sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA==" + }, + "@types/parse5": { + "version": "5.0.3", + "resolved": "https://registry.npmmirror.com/@types/parse5/-/parse5-5.0.3.tgz", + "integrity": "sha512-kUNnecmtkunAoQ3CnjmMkzNU/gtxG8guhi+Fk2U/kOpIKjIMKnXGp4IJCgQJrXSgMsWYimYG4TGjz/UzbGEBTw==" + }, + "@types/prop-types": { + "version": "15.7.5", + "resolved": "https://registry.npmjs.org/@types/prop-types/-/prop-types-15.7.5.tgz", + "integrity": "sha512-JCB8C6SnDoQf0cNycqd/35A7MjcnK+ZTqE7judS6o7utxUCg6imJg3QK2qzHKszlTjcj2cn+NwMB2i96ubpj7w==" + }, + "@types/q": { + "version": "1.5.5", + "resolved": "https://registry.npmjs.org/@types/q/-/q-1.5.5.tgz", + "integrity": "sha512-L28j2FcJfSZOnL1WBjDYp2vUHCeIFlyYI/53EwD/rKUBQ7MtUUfbQWiyKJGpcnv4/WgrhWsFKrcPstcAt/J0tQ==" + }, + "@types/reach__router": { + "version": "1.3.10", + "resolved": "https://registry.npmjs.org/@types/reach__router/-/reach__router-1.3.10.tgz", + "integrity": "sha512-iHAFGaVOrWi00/q7oBybggGsz5TOmwOW4M1H9sT7i9lly4qFC8XOgsdf6jUsoaOz2sknFHALEtZqCoDbokdJ2Q==", + "requires": { + "@types/react": "*" + } + }, + "@types/react": { + "version": "18.0.9", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.0.9.tgz", + "integrity": "sha512-9bjbg1hJHUm4De19L1cHiW0Jvx3geel6Qczhjd0qY5VKVE2X5+x77YxAepuCwVh4vrgZJdgEJw48zrhRIeF4Nw==", + "requires": { + "@types/prop-types": "*", + "@types/scheduler": "*", + "csstype": "^3.0.2" + } + }, + "@types/responselike": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/@types/responselike/-/responselike-1.0.0.tgz", + "integrity": "sha512-85Y2BjiufFzaMIlvJDvTTB8Fxl2xfLo4HgmHzVBz08w4wDePCTjYw66PdrolO0kzli3yam/YCgRufyo1DdQVTA==", + "requires": { + "@types/node": "*" + } + }, + "@types/rimraf": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@types/rimraf/-/rimraf-2.0.5.tgz", + "integrity": "sha512-YyP+VfeaqAyFmXoTh3HChxOQMyjByRMsHU7kc5KOJkSlXudhMhQIALbYV7rHh/l8d2lX3VUQzprrcAgWdRuU8g==", + "requires": { + "@types/glob": "*", + "@types/node": "*" + } + }, + "@types/scheduler": { + "version": "0.16.2", + "resolved": "https://registry.npmjs.org/@types/scheduler/-/scheduler-0.16.2.tgz", + "integrity": "sha512-hppQEBDmlwhFAXKJX2KnWLYu5yMfi91yazPb2l+lbJiwW+wdo1gNeRA+3RgNSO39WYX2euey41KEwnqesU2Jew==" + }, + "@types/sharp": { + "version": "0.30.2", + "resolved": "https://registry.npmjs.org/@types/sharp/-/sharp-0.30.2.tgz", + "integrity": "sha512-uLCBwjDg/BTcQit0dpNGvkIjvH3wsb8zpaJePCjvONBBSfaKHoxXBIuq1MT8DMQEfk2fKYnpC9QExCgFhkGkMQ==", + "requires": { + "@types/node": "*" + } + }, + "@types/tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/@types/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha1-EHPEvIJHVK49EM+riKsCN7qWTk0=" + }, + "@types/unist": { + "version": "2.0.6", + "resolved": "https://registry.npmmirror.com/@types/unist/-/unist-2.0.6.tgz", + "integrity": "sha512-PBjIUxZHOuj0R15/xuwJYjFi+KZdNFrehocChv4g5hu6aFroHue8m0lBP0POdK2nKzbw0cgV1mws8+V/JAcEkQ==" + }, + "@types/websocket": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/@types/websocket/-/websocket-1.0.2.tgz", + "integrity": "sha512-B5m9aq7cbbD/5/jThEr33nUY8WEfVi6A2YKCTOvw5Ldy7mtsOkqRvGjnzy6g7iMMDsgu7xREuCzqATLDLQVKcQ==", + "requires": { + "@types/node": "*" + } + }, + "@types/yoga-layout": { + "version": "1.9.2", + "resolved": "https://registry.npmjs.org/@types/yoga-layout/-/yoga-layout-1.9.2.tgz", + "integrity": "sha512-S9q47ByT2pPvD65IvrWp7qppVMpk9WGMbVq9wbWZOHg6tnXSD4vyhao6nOSBwwfDdV2p3Kx9evA9vI+XWTfDvw==" + }, + "@typescript-eslint/eslint-plugin": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-4.33.0.tgz", + "integrity": "sha512-aINiAxGVdOl1eJyVjaWn/YcVAq4Gi/Yo35qHGCnqbWVz61g39D0h23veY/MA0rFFGfxK7TySg2uwDeNv+JgVpg==", + "requires": { + "@typescript-eslint/experimental-utils": "4.33.0", + "@typescript-eslint/scope-manager": "4.33.0", + "debug": "^4.3.1", + "functional-red-black-tree": "^1.0.1", + "ignore": "^5.1.8", + "regexpp": "^3.1.0", + "semver": "^7.3.5", + "tsutils": "^3.21.0" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "@typescript-eslint/experimental-utils": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/experimental-utils/-/experimental-utils-4.33.0.tgz", + "integrity": "sha512-zeQjOoES5JFjTnAhI5QY7ZviczMzDptls15GFsI6jyUOq0kOf9+WonkhtlIhh0RgHRnqj5gdNxW5j1EvAyYg6Q==", + "requires": { + "@types/json-schema": "^7.0.7", + "@typescript-eslint/scope-manager": "4.33.0", + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/typescript-estree": "4.33.0", + "eslint-scope": "^5.1.1", + "eslint-utils": "^3.0.0" + } + }, + "@typescript-eslint/parser": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-4.33.0.tgz", + "integrity": "sha512-ZohdsbXadjGBSK0/r+d87X0SBmKzOq4/S5nzK6SBgJspFo9/CUDJ7hjayuze+JK7CZQLDMroqytp7pOcFKTxZA==", + "requires": { + "@typescript-eslint/scope-manager": "4.33.0", + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/typescript-estree": "4.33.0", + "debug": "^4.3.1" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "@typescript-eslint/scope-manager": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-4.33.0.tgz", + "integrity": "sha512-5IfJHpgTsTZuONKbODctL4kKuQje/bzBRkwHE8UOZ4f89Zeddg+EGZs8PD8NcN4LdM3ygHWYB3ukPAYjvl/qbQ==", + "requires": { + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/visitor-keys": "4.33.0" + } + }, + "@typescript-eslint/types": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-4.33.0.tgz", + "integrity": "sha512-zKp7CjQzLQImXEpLt2BUw1tvOMPfNoTAfb8l51evhYbOEEzdWyQNmHWWGPR6hwKJDAi+1VXSBmnhL9kyVTTOuQ==" + }, + "@typescript-eslint/typescript-estree": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-4.33.0.tgz", + "integrity": "sha512-rkWRY1MPFzjwnEVHsxGemDzqqddw2QbTJlICPD9p9I9LfsO8fdmfQPOX3uKfUaGRDFJbfrtm/sXhVXN4E+bzCA==", + "requires": { + "@typescript-eslint/types": "4.33.0", + "@typescript-eslint/visitor-keys": "4.33.0", + "debug": "^4.3.1", + "globby": "^11.0.3", + "is-glob": "^4.0.1", + "semver": "^7.3.5", + "tsutils": "^3.21.0" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "@typescript-eslint/visitor-keys": { + "version": "4.33.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-4.33.0.tgz", + "integrity": "sha512-uqi/2aSz9g2ftcHWf8uLPJA70rUv6yuMW5Bohw+bwcuzaxQIHaKFZCKGoGXIrc9vkTJ3+0txM73K0Hq3d5wgIg==", + "requires": { + "@typescript-eslint/types": "4.33.0", + "eslint-visitor-keys": "^2.0.0" + } + }, + "@vercel/webpack-asset-relocator-loader": { + "version": "1.7.2", + "resolved": "https://registry.npmjs.org/@vercel/webpack-asset-relocator-loader/-/webpack-asset-relocator-loader-1.7.2.tgz", + "integrity": "sha512-pdMwUawmAtH/LScbjKJq/y2+gZFggFMc2tlJrlPSrgKajvYPEis3L9QKcMyC9RN1Xos4ezAP5AJfRCNN6RMKCQ==" + }, + "@webassemblyjs/ast": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.1.tgz", + "integrity": "sha512-ukBh14qFLjxTQNTXocdyksN5QdM28S1CxHt2rdskFyL+xFV7VremuBLVbmCePj+URalXBENx/9Lm7lnhihtCSw==", + "requires": { + "@webassemblyjs/helper-numbers": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1" + } + }, + "@webassemblyjs/floating-point-hex-parser": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.1.tgz", + "integrity": "sha512-iGRfyc5Bq+NnNuX8b5hwBrRjzf0ocrJPI6GWFodBFzmFnyvrQ83SHKhmilCU/8Jv67i4GJZBMhEzltxzcNagtQ==" + }, + "@webassemblyjs/helper-api-error": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.1.tgz", + "integrity": "sha512-RlhS8CBCXfRUR/cwo2ho9bkheSXG0+NwooXcc3PAILALf2QLdFyj7KGsKRbVc95hZnhnERon4kW/D3SZpp6Tcg==" + }, + "@webassemblyjs/helper-buffer": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.1.tgz", + "integrity": "sha512-gwikF65aDNeeXa8JxXa2BAk+REjSyhrNC9ZwdT0f8jc4dQQeDQ7G4m0f2QCLPJiMTTO6wfDmRmj/pW0PsUvIcA==" + }, + "@webassemblyjs/helper-numbers": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.1.tgz", + "integrity": "sha512-vDkbxiB8zfnPdNK9Rajcey5C0w+QJugEglN0of+kmO8l7lDb77AnlKYQF7aarZuCrv+l0UvqL+68gSDr3k9LPQ==", + "requires": { + "@webassemblyjs/floating-point-hex-parser": "1.11.1", + "@webassemblyjs/helper-api-error": "1.11.1", + "@xtuc/long": "4.2.2" + } + }, + "@webassemblyjs/helper-wasm-bytecode": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.1.tgz", + "integrity": "sha512-PvpoOGiJwXeTrSf/qfudJhwlvDQxFgelbMqtq52WWiXC6Xgg1IREdngmPN3bs4RoO83PnL/nFrxucXj1+BX62Q==" + }, + "@webassemblyjs/helper-wasm-section": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.1.tgz", + "integrity": "sha512-10P9No29rYX1j7F3EVPX3JvGPQPae+AomuSTPiF9eBQeChHI6iqjMIwR9JmOJXwpnn/oVGDk7I5IlskuMwU/pg==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1" + } + }, + "@webassemblyjs/ieee754": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ieee754/-/ieee754-1.11.1.tgz", + "integrity": "sha512-hJ87QIPtAMKbFq6CGTkZYJivEwZDbQUgYd3qKSadTNOhVY7p+gfP6Sr0lLRVTaG1JjFj+r3YchoqRYxNH3M0GQ==", + "requires": { + "@xtuc/ieee754": "^1.2.0" + } + }, + "@webassemblyjs/leb128": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/leb128/-/leb128-1.11.1.tgz", + "integrity": "sha512-BJ2P0hNZ0u+Th1YZXJpzW6miwqQUGcIHT1G/sf72gLVD9DZ5AdYTqPNbHZh6K1M5VmKvFXwGSWZADz+qBWxeRw==", + "requires": { + "@xtuc/long": "4.2.2" + } + }, + "@webassemblyjs/utf8": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/utf8/-/utf8-1.11.1.tgz", + "integrity": "sha512-9kqcxAEdMhiwQkHpkNiorZzqpGrodQQ2IGrHHxCy+Ozng0ofyMA0lTqiLkVs1uzTRejX+/O0EOT7KxqVPuXosQ==" + }, + "@webassemblyjs/wasm-edit": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.1.tgz", + "integrity": "sha512-g+RsupUC1aTHfR8CDgnsVRVZFJqdkFHpsHMfJuWQzWU3tvnLC07UqHICfP+4XyL2tnr1amvl1Sdp06TnYCmVkA==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/helper-wasm-section": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1", + "@webassemblyjs/wasm-opt": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1", + "@webassemblyjs/wast-printer": "1.11.1" + } + }, + "@webassemblyjs/wasm-gen": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.1.tgz", + "integrity": "sha512-F7QqKXwwNlMmsulj6+O7r4mmtAlCWfO/0HdgOxSklZfQcDu0TpLiD1mRt/zF25Bk59FIjEuGAIyn5ei4yMfLhA==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/ieee754": "1.11.1", + "@webassemblyjs/leb128": "1.11.1", + "@webassemblyjs/utf8": "1.11.1" + } + }, + "@webassemblyjs/wasm-opt": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.1.tgz", + "integrity": "sha512-VqnkNqnZlU5EB64pp1l7hdm3hmQw7Vgqa0KF/KCNO9sIpI6Fk6brDEiX+iCOYrvMuBWDws0NkTOxYEb85XQHHw==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-buffer": "1.11.1", + "@webassemblyjs/wasm-gen": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1" + } + }, + "@webassemblyjs/wasm-parser": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.1.tgz", + "integrity": "sha512-rrBujw+dJu32gYB7/Lup6UhdkPx9S9SnobZzRVL7VcBH9Bt9bCBLEuX/YXOOtBsOZ4NQrRykKhffRWHvigQvOA==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/helper-api-error": "1.11.1", + "@webassemblyjs/helper-wasm-bytecode": "1.11.1", + "@webassemblyjs/ieee754": "1.11.1", + "@webassemblyjs/leb128": "1.11.1", + "@webassemblyjs/utf8": "1.11.1" + } + }, + "@webassemblyjs/wast-printer": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.1.tgz", + "integrity": "sha512-IQboUWM4eKzWW+N/jij2sRatKMh99QEelo3Eb2q0qXkvPRISAj8Qxtmw5itwqK+TTkBuUIE45AxYPToqPtL5gg==", + "requires": { + "@webassemblyjs/ast": "1.11.1", + "@xtuc/long": "4.2.2" + } + }, + "@xtuc/ieee754": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@xtuc/ieee754/-/ieee754-1.2.0.tgz", + "integrity": "sha512-DX8nKgqcGwsc0eJSqYt5lwP4DH5FlHnmuWWBRy7X0NcaGR0ZtuyeESgMwTYVEtxmsNGY+qit4QYT/MIYTOTPeA==" + }, + "@xtuc/long": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/@xtuc/long/-/long-4.2.2.tgz", + "integrity": "sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ==" + }, + "abbrev": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/abbrev/-/abbrev-1.1.1.tgz", + "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" + }, + "abort-controller": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", + "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", + "requires": { + "event-target-shim": "^5.0.0" + } + }, + "abortcontroller-polyfill": { + "version": "1.7.3", + "resolved": "https://registry.npmjs.org/abortcontroller-polyfill/-/abortcontroller-polyfill-1.7.3.tgz", + "integrity": "sha512-zetDJxd89y3X99Kvo4qFx8GKlt6GsvN3UcRZHwU6iFA/0KiOmhkTVhe8oRoTBiTVPZu09x3vCra47+w8Yz1+2Q==" + }, + "accepts": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/accepts/-/accepts-1.3.8.tgz", + "integrity": "sha512-PYAthTa2m2VKxuvSD3DPC/Gy+U+sOA1LAuT8mkmRuvw+NACSaeXEQ+NHcVF7rONl6qcaxV3Uuemwawk+7+SJLw==", + "requires": { + "mime-types": "~2.1.34", + "negotiator": "0.6.3" + } + }, + "acorn": { + "version": "7.4.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-7.4.1.tgz", + "integrity": "sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A==" + }, + "acorn-import-assertions": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.8.0.tgz", + "integrity": "sha512-m7VZ3jwz4eK6A4Vtt8Ew1/mNbP24u0FhdyfA7fSvnJR6LMdfOYnmuIrrJAgrYfYJ10F/otaHTtrtrtmHdMNzEw==", + "requires": {} + }, + "acorn-jsx": { + "version": "5.3.2", + "resolved": "https://registry.npmjs.org/acorn-jsx/-/acorn-jsx-5.3.2.tgz", + "integrity": "sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ==", + "requires": {} + }, + "address": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/address/-/address-1.1.2.tgz", + "integrity": "sha512-aT6camzM4xEA54YVJYSqxz1kv4IHnQZRtThJJHhUMRExaU5spC7jX5ugSwTaTgJliIgs4VhZOk7htClvQ/LmRA==" + }, + "adjust-sourcemap-loader": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/adjust-sourcemap-loader/-/adjust-sourcemap-loader-3.0.0.tgz", + "integrity": "sha512-YBrGyT2/uVQ/c6Rr+t6ZJXniY03YtHGMJQYal368burRGYKqhx9qGTWqcBU5s1CwYY9E/ri63RYyG1IacMZtqw==", + "requires": { + "loader-utils": "^2.0.0", + "regex-parser": "^2.2.11" + } + }, + "ajv": { + "version": "6.12.6", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.12.6.tgz", + "integrity": "sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==", + "requires": { + "fast-deep-equal": "^3.1.1", + "fast-json-stable-stringify": "^2.0.0", + "json-schema-traverse": "^0.4.1", + "uri-js": "^4.2.2" + } + }, + "ajv-keywords": { + "version": "3.5.2", + "resolved": "https://registry.npmjs.org/ajv-keywords/-/ajv-keywords-3.5.2.tgz", + "integrity": "sha512-5p6WTN0DdTGVQk6VjcEju19IgaHudalcfabD7yhDGeA6bcQnmL+CpveLJq/3hvfwd1aof6L386Ougkx6RfyMIQ==", + "requires": {} + }, + "amdefine": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/amdefine/-/amdefine-1.0.1.tgz", + "integrity": "sha512-S2Hw0TtNkMJhIabBwIojKL9YHO5T0n5eNqWJ7Lrlel/zDbftQpxpapi8tZs3X1HWa+u+QeydGmzzNU0m09+Rcg==" + }, + "anser": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/anser/-/anser-2.1.1.tgz", + "integrity": "sha512-nqLm4HxOTpeLOxcmB3QWmV5TcDFhW9y/fyQ+hivtDFcK4OQ+pQ5fzPnXHM1Mfcm0VkLtvVi1TCPr++Qy0Q/3EQ==" + }, + "ansi-align": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/ansi-align/-/ansi-align-3.0.1.tgz", + "integrity": "sha512-IOfwwBF5iczOjp/WeY4YxyjqAFMQoZufdQWDd19SEExbVLNXqvpzSJ/M7Za4/sCPmQ0+GRquoA7bGcINcxew6w==", + "requires": { + "string-width": "^4.1.0" + } + }, + "ansi-colors": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", + "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==" + }, + "ansi-escapes": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-3.2.0.tgz", + "integrity": "sha512-cBhpre4ma+U0T1oM5fXg7Dy1Jw7zzwv7lt/GoCpr+hDQJoYnKVPLL4dCvSEFMmQurOQvSrwT7SL/DAlhBI97RQ==" + }, + "ansi-html-community": { + "version": "0.0.8", + "resolved": "https://registry.npmjs.org/ansi-html-community/-/ansi-html-community-0.0.8.tgz", + "integrity": "sha512-1APHAyr3+PCamwNw3bXCPp4HFLONZt/yIH0sZp0/469KWNTEy+qN5jQ3GVX6DMZ1UXAi34yVwtTeaG/HpBuuzw==" + }, + "ansi-regex": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", + "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==" + }, + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "requires": { + "color-convert": "^1.9.0" + } + }, + "any-base": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/any-base/-/any-base-1.1.0.tgz", + "integrity": "sha512-uMgjozySS8adZZYePpaWs8cxB9/kdzmpX6SgJZ+wbz1K5eYk5QMYDVJaZKhxyIHUdnnJkfR7SVgStgH7LkGUyg==" + }, + "anymatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.2.tgz", + "integrity": "sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==", + "requires": { + "normalize-path": "^3.0.0", + "picomatch": "^2.0.4" + } + }, + "append-field": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/append-field/-/append-field-1.0.0.tgz", + "integrity": "sha1-HjRA6RXwsSA9I3SOeO3XubW0PlY=" + }, + "application-config-path": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/application-config-path/-/application-config-path-0.1.0.tgz", + "integrity": "sha1-GTxfCoZUGkxm+6Hi3DhYM2LqXo8=" + }, + "aproba": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/aproba/-/aproba-1.2.0.tgz", + "integrity": "sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw==" + }, + "arch": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/arch/-/arch-2.2.0.tgz", + "integrity": "sha512-Of/R0wqp83cgHozfIYLbBMnej79U/SVGOOyuB3VVFv1NRM/PSFMK12x9KVtiYzJqmnU5WR2qp0Z5rHb7sWGnFQ==" + }, + "are-we-there-yet": { + "version": "1.1.7", + "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-1.1.7.tgz", + "integrity": "sha512-nxwy40TuMiUGqMyRHgCSWZ9FM4VAoRP4xUYSTv5ImRog+h9yISPbVH7H8fASCIzYn9wlEv4zvFL7uKDMCFQm3g==", + "requires": { + "delegates": "^1.0.0", + "readable-stream": "^2.0.6" + }, + "dependencies": { + "readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "requires": { + "safe-buffer": "~5.1.0" + } + } + } + }, + "arg": { + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/arg/-/arg-4.1.3.tgz", + "integrity": "sha512-58S9QDqG0Xx27YwPSt9fJxivjYl432YCwfDMfZ+71RAqUrZef7LrKQZ3LHLOwCS4FLNBplP533Zx895SeOCHvA==" + }, + "argparse": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", + "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", + "requires": { + "sprintf-js": "~1.0.2" + } + }, + "aria-query": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/aria-query/-/aria-query-4.2.2.tgz", + "integrity": "sha512-o/HelwhuKpTj/frsOsbNLNgnNGVIFsVP/SW2BSF14gVl7kAfMOJ6/8wUAUvG1R1NHKrfG+2sHZTu0yauT1qBrA==", + "requires": { + "@babel/runtime": "^7.10.2", + "@babel/runtime-corejs3": "^7.10.2" + } + }, + "arity-n": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/arity-n/-/arity-n-1.0.4.tgz", + "integrity": "sha512-fExL2kFDC1Q2DUOx3whE/9KoN66IzkY4b4zUHUBFM1ojEYjZZYDcUW3bek/ufGionX9giIKDC5redH2IlGqcQQ==" + }, + "array-flatten": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-1.1.1.tgz", + "integrity": "sha1-ml9pkFGx5wczKPKgCJaLZOopVdI=" + }, + "array-includes": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/array-includes/-/array-includes-3.1.5.tgz", + "integrity": "sha512-iSDYZMMyTPkiFasVqfuAQnWAYcvO/SeBSCGKePoEthjp4LEMTe4uLc7b025o4jAZpHhihh8xPo99TNWUWWkGDQ==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5", + "get-intrinsic": "^1.1.1", + "is-string": "^1.0.7" + } + }, + "array-iterate": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/array-iterate/-/array-iterate-1.1.4.tgz", + "integrity": "sha512-sNRaPGh9nnmdC8Zf+pT3UqP8rnWj5Hf9wiFGsX3wUQ2yVSIhO2ShFwCoceIPpB41QF6i2OEmrHmCo36xronCVA==" + }, + "array-union": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/array-union/-/array-union-2.1.0.tgz", + "integrity": "sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==" + }, + "array.prototype.flat": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/array.prototype.flat/-/array.prototype.flat-1.3.0.tgz", + "integrity": "sha512-12IUEkHsAhA4DY5s0FPgNXIdc8VRSqD9Zp78a5au9abH/SOBrsp082JOWFNTjkMozh8mqcdiKuaLGhPeYztxSw==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.2", + "es-shim-unscopables": "^1.0.0" + } + }, + "array.prototype.flatmap": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/array.prototype.flatmap/-/array.prototype.flatmap-1.3.0.tgz", + "integrity": "sha512-PZC9/8TKAIxcWKdyeb77EzULHPrIX/tIZebLJUQOMR1OwYosT8yggdfWScfTBCDj5utONvOuPQQumYsU2ULbkg==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.2", + "es-shim-unscopables": "^1.0.0" + } + }, + "arrify": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/arrify/-/arrify-2.0.1.tgz", + "integrity": "sha512-3duEwti880xqi4eAMN8AyR4a0ByT90zoYdLlevfrvU43vb0YZwZVfxOgxWrLXXXpyugL0hNZc9G6BiB5B3nUug==" + }, + "asap": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/asap/-/asap-2.0.6.tgz", + "integrity": "sha1-5QNHYR1+aQlDIIu9r+vLwvuGbUY=" + }, + "asn1": { + "version": "0.2.6", + "resolved": "https://registry.npmmirror.com/asn1/-/asn1-0.2.6.tgz", + "integrity": "sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==", + "requires": { + "safer-buffer": "~2.1.0" + } + }, + "assert-plus": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/assert-plus/-/assert-plus-1.0.0.tgz", + "integrity": "sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==" + }, + "ast-types-flow": { + "version": "0.0.7", + "resolved": "https://registry.npmjs.org/ast-types-flow/-/ast-types-flow-0.0.7.tgz", + "integrity": "sha1-9wtzXGvKGlycItmCw+Oef+ujva0=" + }, + "astral-regex": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/astral-regex/-/astral-regex-2.0.0.tgz", + "integrity": "sha512-Z7tMw1ytTXt5jqMcOP+OQteU1VuNK9Y02uuJtKQ1Sv69jXQKKg5cibLwGJow8yzZP+eAc18EmLGPal0bp36rvQ==" + }, + "async": { + "version": "1.5.2", + "resolved": "https://registry.npmjs.org/async/-/async-1.5.2.tgz", + "integrity": "sha1-7GphrlZIDAw8skHJVhjiCJL5Zyo=" + }, + "async-cache": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/async-cache/-/async-cache-1.1.0.tgz", + "integrity": "sha1-SppaidBl7F2OUlS9nulrp2xTK1o=", + "requires": { + "lru-cache": "^4.0.0" + }, + "dependencies": { + "lru-cache": { + "version": "4.1.5", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-4.1.5.tgz", + "integrity": "sha512-sWZlbEP2OsHNkXrMl5GYk/jKk70MBng6UU4YI/qGDYbgf6YbP4EvmqISbXCoJiRKs+1bSpFHVgQxvJ17F2li5g==", + "requires": { + "pseudomap": "^1.0.2", + "yallist": "^2.1.2" + } + }, + "yallist": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-2.1.2.tgz", + "integrity": "sha1-HBH5IY8HYImkfdUS+TxmmaaoHVI=" + } + } + }, + "async-foreach": { + "version": "0.1.3", + "resolved": "https://registry.npmmirror.com/async-foreach/-/async-foreach-0.1.3.tgz", + "integrity": "sha512-VUeSMD8nEGBWaZK4lizI1sf3yEC7pnAQ/mrI7pC2fBz2s/tq5jWWEngTwaf0Gruu/OoXRGLGg1XFqpYBiGTYJA==" + }, + "async-retry-ng": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/async-retry-ng/-/async-retry-ng-2.0.1.tgz", + "integrity": "sha512-iitlc2murdQ3/A5Re3CcplQBEf7vOmFrFQ6RFn3+/+zZUyIHYkZnnEziMSa6YIb2Bs2EJEPZWReTxjHqvQbDbw==" + }, + "asynckit": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz", + "integrity": "sha1-x57Zf380y48robyXkLzDZkdLS3k=" + }, + "at-least-node": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/at-least-node/-/at-least-node-1.0.0.tgz", + "integrity": "sha512-+q/t7Ekv1EDY2l6Gda6LLiX14rU9TV20Wa3ofeQmwPFZbOMo9DXrLbOjFaaclkXKWidIaopwAObQDqwWtGUjqg==" + }, + "atob": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/atob/-/atob-2.1.2.tgz", + "integrity": "sha512-Wm6ukoaOGJi/73p/cl2GvLjTI5JM1k/O14isD73YML8StrH/7/lRFgmg8nICZgD3bZZvjwCGxtMOD3wWNAu8cg==" + }, + "auto-bind": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/auto-bind/-/auto-bind-4.0.0.tgz", + "integrity": "sha512-Hdw8qdNiqdJ8LqT0iK0sVzkFbzg6fhnQqqfWhBDxcHZvU75+B+ayzTy8x+k5Ix0Y92XOhOUlx74ps+bA6BeYMQ==" + }, + "autoprefixer": { + "version": "10.4.7", + "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-10.4.7.tgz", + "integrity": "sha512-ypHju4Y2Oav95SipEcCcI5J7CGPuvz8oat7sUtYj3ClK44bldfvtvcxK6IEK++7rqB7YchDGzweZIBG+SD0ZAA==", + "requires": { + "browserslist": "^4.20.3", + "caniuse-lite": "^1.0.30001335", + "fraction.js": "^4.2.0", + "normalize-range": "^0.1.2", + "picocolors": "^1.0.0", + "postcss-value-parser": "^4.2.0" + } + }, + "aws-sign2": { + "version": "0.7.0", + "resolved": "https://registry.npmmirror.com/aws-sign2/-/aws-sign2-0.7.0.tgz", + "integrity": "sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==" + }, + "aws4": { + "version": "1.11.0", + "resolved": "https://registry.npmmirror.com/aws4/-/aws4-1.11.0.tgz", + "integrity": "sha512-xh1Rl34h6Fi1DC2WWKfxUTVqRsNnr6LsKz2+hfwDxQJWmrx8+c7ylaqBMcHfl1U1r2dsifOvKX3LQuLNZ+XSvA==" + }, + "axe-core": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/axe-core/-/axe-core-4.4.1.tgz", + "integrity": "sha512-gd1kmb21kwNuWr6BQz8fv6GNECPBnUasepcoLbekws23NVBLODdsClRZ+bQ8+9Uomf3Sm3+Vwn0oYG9NvwnJCw==" + }, + "axios": { + "version": "0.21.4", + "resolved": "https://registry.npmjs.org/axios/-/axios-0.21.4.tgz", + "integrity": "sha512-ut5vewkiu8jjGBdqpM44XxjuCjq9LAKeHVmoVfHVzy8eHgxxq8SbAVQNovDA8mVi05kP0Ea/n/UzcSHcTJQfNg==", + "requires": { + "follow-redirects": "^1.14.0" + } + }, + "axobject-query": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/axobject-query/-/axobject-query-2.2.0.tgz", + "integrity": "sha512-Td525n+iPOOyUQIeBfcASuG6uJsDOITl7Mds5gFyerkWiX7qhUTdYUBlSgNMyVqtSJqwpt1kXGLdUt6SykLMRA==" + }, + "babel-extract-comments": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/babel-extract-comments/-/babel-extract-comments-1.0.0.tgz", + "integrity": "sha512-qWWzi4TlddohA91bFwgt6zO/J0X+io7Qp184Fw0m2JYRSTZnJbFR8+07KmzudHCZgOiKRCrjhylwv9Xd8gfhVQ==", + "requires": { + "babylon": "^6.18.0" + } + }, + "babel-jsx-utils": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/babel-jsx-utils/-/babel-jsx-utils-1.1.0.tgz", + "integrity": "sha512-Mh1j/rw4xM9T3YICkw22aBQ78FhsHdsmlb9NEk4uVAFBOg+Ez9ZgXXHugoBPCZui3XLomk/7/JBBH4daJqTkQQ==" + }, + "babel-loader": { + "version": "8.2.5", + "resolved": "https://registry.npmjs.org/babel-loader/-/babel-loader-8.2.5.tgz", + "integrity": "sha512-OSiFfH89LrEMiWd4pLNqGz4CwJDtbs2ZVc+iGu2HrkRfPxId9F2anQj38IxWpmRfsUY0aBZYi1EFcd3mhtRMLQ==", + "requires": { + "find-cache-dir": "^3.3.1", + "loader-utils": "^2.0.0", + "make-dir": "^3.1.0", + "schema-utils": "^2.6.5" + } + }, + "babel-plugin-add-module-exports": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/babel-plugin-add-module-exports/-/babel-plugin-add-module-exports-1.0.4.tgz", + "integrity": "sha512-g+8yxHUZ60RcyaUpfNzy56OtWW+x9cyEe9j+CranqLiqbju2yf/Cy6ZtYK40EZxtrdHllzlVZgLmcOUCTlJ7Jg==" + }, + "babel-plugin-dynamic-import-node": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/babel-plugin-dynamic-import-node/-/babel-plugin-dynamic-import-node-2.3.3.tgz", + "integrity": "sha512-jZVI+s9Zg3IqA/kdi0i6UDCybUI3aSBLnglhYbSSjKlV7yF1F/5LWv8MakQmvYpnbJDS6fcBL2KzHSxNCMtWSQ==", + "requires": { + "object.assign": "^4.1.0" + } + }, + "babel-plugin-lodash": { + "version": "3.3.4", + "resolved": "https://registry.npmjs.org/babel-plugin-lodash/-/babel-plugin-lodash-3.3.4.tgz", + "integrity": "sha512-yDZLjK7TCkWl1gpBeBGmuaDIFhZKmkoL+Cu2MUUjv5VxUZx/z7tBGBCBcQs5RI1Bkz5LLmNdjx7paOyQtMovyg==", + "requires": { + "@babel/helper-module-imports": "^7.0.0-beta.49", + "@babel/types": "^7.0.0-beta.49", + "glob": "^7.1.1", + "lodash": "^4.17.10", + "require-package-name": "^2.0.1" + } + }, + "babel-plugin-macros": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/babel-plugin-macros/-/babel-plugin-macros-3.1.0.tgz", + "integrity": "sha512-Cg7TFGpIr01vOQNODXOOaGz2NpCU5gl8x1qJFbb6hbZxR7XrcE2vtbAsTAbJ7/xwJtUuJEw8K8Zr/AE0LHlesg==", + "requires": { + "@babel/runtime": "^7.12.5", + "cosmiconfig": "^7.0.0", + "resolve": "^1.19.0" + } + }, + "babel-plugin-polyfill-corejs2": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.3.1.tgz", + "integrity": "sha512-v7/T6EQcNfVLfcN2X8Lulb7DjprieyLWJK/zOWH5DUYcAgex9sP3h25Q+DLsX9TloXe3y1O8l2q2Jv9q8UVB9w==", + "requires": { + "@babel/compat-data": "^7.13.11", + "@babel/helper-define-polyfill-provider": "^0.3.1", + "semver": "^6.1.1" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "babel-plugin-polyfill-corejs3": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.5.2.tgz", + "integrity": "sha512-G3uJih0XWiID451fpeFaYGVuxHEjzKTHtc9uGFEjR6hHrvNzeS/PX+LLLcetJcytsB5m4j+K3o/EpXJNb/5IEQ==", + "requires": { + "@babel/helper-define-polyfill-provider": "^0.3.1", + "core-js-compat": "^3.21.0" + } + }, + "babel-plugin-polyfill-regenerator": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.3.1.tgz", + "integrity": "sha512-Y2B06tvgHYt1x0yz17jGkGeeMr5FeKUu+ASJ+N6nB5lQ8Dapfg42i0OVrf8PNGJ3zKL4A23snMi1IRwrqqND7A==", + "requires": { + "@babel/helper-define-polyfill-provider": "^0.3.1" + } + }, + "babel-plugin-remove-graphql-queries": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/babel-plugin-remove-graphql-queries/-/babel-plugin-remove-graphql-queries-4.14.0.tgz", + "integrity": "sha512-rqCih6maArH0nbkndAP9UKKQCUWZy1NBxG+nSOoIZpvLkMqTweAuiTpMDJVHWDk9CycFlLfl09/Ayk/nciVKhA==", + "requires": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0" + } + }, + "babel-plugin-syntax-object-rest-spread": { + "version": "6.13.0", + "resolved": "https://registry.npmjs.org/babel-plugin-syntax-object-rest-spread/-/babel-plugin-syntax-object-rest-spread-6.13.0.tgz", + "integrity": "sha1-/WU28rzhODb/o6VFjEkDpZe7O/U=" + }, + "babel-plugin-syntax-trailing-function-commas": { + "version": "7.0.0-beta.0", + "resolved": "https://registry.npmjs.org/babel-plugin-syntax-trailing-function-commas/-/babel-plugin-syntax-trailing-function-commas-7.0.0-beta.0.tgz", + "integrity": "sha512-Xj9XuRuz3nTSbaTXWv3itLOcxyF4oPD8douBBmj7U9BBC6nEBYfyOJYQMf/8PJAFotC62UY5dFfIGEPr7WswzQ==" + }, + "babel-plugin-transform-object-rest-spread": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-plugin-transform-object-rest-spread/-/babel-plugin-transform-object-rest-spread-6.26.0.tgz", + "integrity": "sha1-DzZpLVD+9rfi1LOsFHgTepY7ewY=", + "requires": { + "babel-plugin-syntax-object-rest-spread": "^6.8.0", + "babel-runtime": "^6.26.0" + } + }, + "babel-plugin-transform-react-remove-prop-types": { + "version": "0.4.24", + "resolved": "https://registry.npmjs.org/babel-plugin-transform-react-remove-prop-types/-/babel-plugin-transform-react-remove-prop-types-0.4.24.tgz", + "integrity": "sha512-eqj0hVcJUR57/Ug2zE1Yswsw4LhuqqHhD+8v120T1cl3kjg76QwtyBrdIk4WVwK+lAhBJVYCd/v+4nc4y+8JsA==" + }, + "babel-preset-fbjs": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/babel-preset-fbjs/-/babel-preset-fbjs-3.4.0.tgz", + "integrity": "sha512-9ywCsCvo1ojrw0b+XYk7aFvTH6D9064t0RIL1rtMf3nsa02Xw41MS7sZw216Im35xj/UY0PDBQsa1brUDDF1Ow==", + "requires": { + "@babel/plugin-proposal-class-properties": "^7.0.0", + "@babel/plugin-proposal-object-rest-spread": "^7.0.0", + "@babel/plugin-syntax-class-properties": "^7.0.0", + "@babel/plugin-syntax-flow": "^7.0.0", + "@babel/plugin-syntax-jsx": "^7.0.0", + "@babel/plugin-syntax-object-rest-spread": "^7.0.0", + "@babel/plugin-transform-arrow-functions": "^7.0.0", + "@babel/plugin-transform-block-scoped-functions": "^7.0.0", + "@babel/plugin-transform-block-scoping": "^7.0.0", + "@babel/plugin-transform-classes": "^7.0.0", + "@babel/plugin-transform-computed-properties": "^7.0.0", + "@babel/plugin-transform-destructuring": "^7.0.0", + "@babel/plugin-transform-flow-strip-types": "^7.0.0", + "@babel/plugin-transform-for-of": "^7.0.0", + "@babel/plugin-transform-function-name": "^7.0.0", + "@babel/plugin-transform-literals": "^7.0.0", + "@babel/plugin-transform-member-expression-literals": "^7.0.0", + "@babel/plugin-transform-modules-commonjs": "^7.0.0", + "@babel/plugin-transform-object-super": "^7.0.0", + "@babel/plugin-transform-parameters": "^7.0.0", + "@babel/plugin-transform-property-literals": "^7.0.0", + "@babel/plugin-transform-react-display-name": "^7.0.0", + "@babel/plugin-transform-react-jsx": "^7.0.0", + "@babel/plugin-transform-shorthand-properties": "^7.0.0", + "@babel/plugin-transform-spread": "^7.0.0", + "@babel/plugin-transform-template-literals": "^7.0.0", + "babel-plugin-syntax-trailing-function-commas": "^7.0.0-beta.0" + } + }, + "babel-preset-gatsby": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/babel-preset-gatsby/-/babel-preset-gatsby-2.14.0.tgz", + "integrity": "sha512-IqPgd15jJfJvqvX0i78JwLT48ctb7MdIEqHeKOuo4N8qWmyRIY8xX1IVhhSfDZ3eq62j0rVoqzT7ACUWctikmw==", + "requires": { + "@babel/plugin-proposal-class-properties": "^7.14.0", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.14.5", + "@babel/plugin-proposal-optional-chaining": "^7.14.5", + "@babel/plugin-syntax-dynamic-import": "^7.8.3", + "@babel/plugin-transform-classes": "^7.15.4", + "@babel/plugin-transform-runtime": "^7.15.0", + "@babel/plugin-transform-spread": "^7.14.6", + "@babel/preset-env": "^7.15.4", + "@babel/preset-react": "^7.14.0", + "@babel/runtime": "^7.15.4", + "babel-plugin-dynamic-import-node": "^2.3.3", + "babel-plugin-macros": "^3.1.0", + "babel-plugin-transform-react-remove-prop-types": "^0.4.24", + "gatsby-core-utils": "^3.14.0", + "gatsby-legacy-polyfills": "^2.14.0" + } + }, + "babel-runtime": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-runtime/-/babel-runtime-6.26.0.tgz", + "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=", + "requires": { + "core-js": "^2.4.0", + "regenerator-runtime": "^0.11.0" + }, + "dependencies": { + "core-js": { + "version": "2.6.12", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.6.12.tgz", + "integrity": "sha512-Kb2wC0fvsWfQrgk8HU5lW6U/Lcs8+9aaYcy4ZFc6DDlo4nZ7n70dEgE5rtR0oG6ufKDUnrwfWL1mXR5ljDatrQ==" + }, + "regenerator-runtime": { + "version": "0.11.1", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.11.1.tgz", + "integrity": "sha512-MguG95oij0fC3QV3URf4V2SDYGJhJnJGqvIIgdECeODCT98wSWDAJ94SSuVpYQUoTcGUIL6L4yNB7j1DFFHSBg==" + } + } + }, + "babylon": { + "version": "6.18.0", + "resolved": "https://registry.npmjs.org/babylon/-/babylon-6.18.0.tgz", + "integrity": "sha512-q/UEjfGJ2Cm3oKV71DJz9d25TPnq5rhBVL2Q4fA5wcC3jcrdn7+SssEybFIxwAvvP+YCsCYNKughoF33GxgycQ==" + }, + "backo2": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/backo2/-/backo2-1.0.2.tgz", + "integrity": "sha1-MasayLEpNjRj41s+u2n038+6eUc=" + }, + "bail": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/bail/-/bail-1.0.5.tgz", + "integrity": "sha512-xFbRxM1tahm08yHBP16MMjVUAvDaBMD38zsM9EMAUN61omwLmKlOpB/Zku5QkjZ8TZ4vn53pj+t518cH0S03RQ==" + }, + "balanced-match": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + }, + "base-x": { + "version": "3.0.9", + "resolved": "https://registry.npmjs.org/base-x/-/base-x-3.0.9.tgz", + "integrity": "sha512-H7JU6iBHTal1gp56aKoaa//YUxEaAOUiydvrV/pILqIHXTtqxSkATOnDA2u+jZ/61sD+L/412+7kzXRtWukhpQ==", + "requires": { + "safe-buffer": "^5.0.1" + } + }, + "base64-arraybuffer": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/base64-arraybuffer/-/base64-arraybuffer-0.1.4.tgz", + "integrity": "sha1-mBjHngWbE1X5fgQooBfIOOkLqBI=" + }, + "base64-js": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz", + "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==" + }, + "base64id": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/base64id/-/base64id-2.0.0.tgz", + "integrity": "sha512-lGe34o6EHj9y3Kts9R4ZYs/Gr+6N7MCaMlIFA3F1R2O5/m7K06AxfSeO5530PEERE6/WyEg3lsuyw4GHlPZHog==" + }, + "bcrypt-pbkdf": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz", + "integrity": "sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==", + "requires": { + "tweetnacl": "^0.14.3" + } + }, + "better-opn": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/better-opn/-/better-opn-2.1.1.tgz", + "integrity": "sha512-kIPXZS5qwyKiX/HcRvDYfmBQUa8XP17I0mYZZ0y4UhpYOSvtsLHDYqmomS+Mj20aDvD3knEiQ0ecQy2nhio3yA==", + "requires": { + "open": "^7.0.3" + } + }, + "big.js": { + "version": "5.2.2", + "resolved": "https://registry.npmjs.org/big.js/-/big.js-5.2.2.tgz", + "integrity": "sha512-vyL2OymJxmarO8gxMr0mhChsO9QGwhynfuu4+MHTAW6czfq9humCB7rKpUjDd9YUiDPU4mzpyupFSvOClAwbmQ==" + }, + "binary-extensions": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", + "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==" + }, + "bl": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/bl/-/bl-4.1.0.tgz", + "integrity": "sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w==", + "requires": { + "buffer": "^5.5.0", + "inherits": "^2.0.4", + "readable-stream": "^3.4.0" + } + }, + "bluebird": { + "version": "3.7.2", + "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz", + "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==" + }, + "bmp-js": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/bmp-js/-/bmp-js-0.1.0.tgz", + "integrity": "sha1-4Fpj95amwf8l9Hcex62twUjAcjM=" + }, + "body-parser": { + "version": "1.20.0", + "resolved": "https://registry.npmjs.org/body-parser/-/body-parser-1.20.0.tgz", + "integrity": "sha512-DfJ+q6EPcGKZD1QWUjSpqp+Q7bDQTsQIF4zfUAtZ6qk+H/3/QRhg9CEp39ss+/T2vw0+HaidC0ecJj/DRLIaKg==", + "requires": { + "bytes": "3.1.2", + "content-type": "~1.0.4", + "debug": "2.6.9", + "depd": "2.0.0", + "destroy": "1.2.0", + "http-errors": "2.0.0", + "iconv-lite": "0.4.24", + "on-finished": "2.4.1", + "qs": "6.10.3", + "raw-body": "2.5.1", + "type-is": "~1.6.18", + "unpipe": "1.0.0" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "boolbase": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/boolbase/-/boolbase-1.0.0.tgz", + "integrity": "sha1-aN/1++YMUes3cl6p4+0xDcwed24=" + }, + "boxen": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/boxen/-/boxen-5.1.2.tgz", + "integrity": "sha512-9gYgQKXx+1nP8mP7CzFyaUARhg7D3n1dF/FnErWmu9l6JvGpNUN278h0aSb+QjoiKSWG+iZ3uHrcqk0qrY9RQQ==", + "requires": { + "ansi-align": "^3.0.0", + "camelcase": "^6.2.0", + "chalk": "^4.1.0", + "cli-boxes": "^2.2.1", + "string-width": "^4.2.2", + "type-fest": "^0.20.2", + "widest-line": "^3.1.0", + "wrap-ansi": "^7.0.0" + }, + "dependencies": { + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "requires": { + "color-convert": "^2.0.1" + } + }, + "camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==" + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "wrap-ansi": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "requires": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + } + } + } + }, + "brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "braces": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", + "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "requires": { + "fill-range": "^7.0.1" + } + }, + "browserslist": { + "version": "4.20.3", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.20.3.tgz", + "integrity": "sha512-NBhymBQl1zM0Y5dQT/O+xiLP9/rzOIQdKM/eMJBAq7yBgaB6krIYLGejrwVYnSHZdqjscB1SPuAjHwxjvN6Wdg==", + "requires": { + "caniuse-lite": "^1.0.30001332", + "electron-to-chromium": "^1.4.118", + "escalade": "^3.1.1", + "node-releases": "^2.0.3", + "picocolors": "^1.0.0" + } + }, + "bser": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/bser/-/bser-2.1.1.tgz", + "integrity": "sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ==", + "requires": { + "node-int64": "^0.4.0" + } + }, + "buffer": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/buffer/-/buffer-5.7.1.tgz", + "integrity": "sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ==", + "requires": { + "base64-js": "^1.3.1", + "ieee754": "^1.1.13" + } + }, + "buffer-equal": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/buffer-equal/-/buffer-equal-0.0.1.tgz", + "integrity": "sha1-kbx0sR6kBbyRa8aqkI+q+ltKrEs=" + }, + "buffer-from": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/buffer-from/-/buffer-from-1.1.2.tgz", + "integrity": "sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==" + }, + "busboy": { + "version": "0.2.14", + "resolved": "https://registry.npmjs.org/busboy/-/busboy-0.2.14.tgz", + "integrity": "sha1-bCpiLvz0fFe7vh4qnDetNseSVFM=", + "requires": { + "dicer": "0.2.5", + "readable-stream": "1.1.x" + }, + "dependencies": { + "isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8=" + }, + "readable-stream": { + "version": "1.1.14", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", + "integrity": "sha1-fPTFTvZI44EwhMY23SB54WbAgdk=", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=" + } + } + }, + "bytes": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/bytes/-/bytes-3.1.2.tgz", + "integrity": "sha512-/Nf7TyzTx6S3yRJObOAV7956r8cr2+Oj8AC5dt8wSP3BQAoeX58NoHyCU8P8zGkNXStjTSi6fzO6F0pBdcYbEg==" + }, + "cache-manager": { + "version": "2.11.1", + "resolved": "https://registry.npmjs.org/cache-manager/-/cache-manager-2.11.1.tgz", + "integrity": "sha512-XhUuc9eYwkzpK89iNewFwtvcDYMUsvtwzHeyEOPJna/WsVsXcrzsA1ft2M0QqPNunEzLhNCYPo05tEfG+YuNow==", + "requires": { + "async": "1.5.2", + "lodash.clonedeep": "4.5.0", + "lru-cache": "4.0.0" + }, + "dependencies": { + "lru-cache": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-4.0.0.tgz", + "integrity": "sha1-tcvwFVbBaWb+vlTO7A+03JDfbCg=", + "requires": { + "pseudomap": "^1.0.1", + "yallist": "^2.0.0" + } + }, + "yallist": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-2.1.2.tgz", + "integrity": "sha1-HBH5IY8HYImkfdUS+TxmmaaoHVI=" + } + } + }, + "cacheable-lookup": { + "version": "5.0.4", + "resolved": "https://registry.npmjs.org/cacheable-lookup/-/cacheable-lookup-5.0.4.tgz", + "integrity": "sha512-2/kNscPhpcxrOigMZzbiWF7dz8ilhb/nIHU3EyZiXWXpeq/au8qJ8VhdftMkty3n7Gj6HIGalQG8oiBNB3AJgA==" + }, + "cacheable-request": { + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-7.0.2.tgz", + "integrity": "sha512-pouW8/FmiPQbuGpkXQ9BAPv/Mo5xDGANgSNXzTzJ8DrKGuXOssM4wIQRjfanNRh3Yu5cfYPvcorqbhg2KIJtew==", + "requires": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^4.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^6.0.1", + "responselike": "^2.0.0" + } + }, + "call-bind": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/call-bind/-/call-bind-1.0.2.tgz", + "integrity": "sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA==", + "requires": { + "function-bind": "^1.1.1", + "get-intrinsic": "^1.0.2" + } + }, + "callsites": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/callsites/-/callsites-3.1.0.tgz", + "integrity": "sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ==" + }, + "camel-case": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/camel-case/-/camel-case-4.1.2.tgz", + "integrity": "sha512-gxGWBrTT1JuMx6R+o5PTXMmUnhnVzLQ9SNutD4YqKtI6ap897t3tKECYla6gCWEkplXnlNybEkZg9GEGxKFCgw==", + "requires": { + "pascal-case": "^3.1.2", + "tslib": "^2.0.3" + } + }, + "camelcase": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.3.1.tgz", + "integrity": "sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==" + }, + "camelcase-keys": { + "version": "6.2.2", + "resolved": "https://registry.npmmirror.com/camelcase-keys/-/camelcase-keys-6.2.2.tgz", + "integrity": "sha512-YrwaA0vEKazPBkn0ipTiMpSajYDSe+KjQfrjhcBMxJt/znbvlHd8Pw/Vamaz5EB4Wfhs3SUR3Z9mwRu/P3s3Yg==", + "requires": { + "camelcase": "^5.3.1", + "map-obj": "^4.0.0", + "quick-lru": "^4.0.1" + }, + "dependencies": { + "quick-lru": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/quick-lru/-/quick-lru-4.0.1.tgz", + "integrity": "sha512-ARhCpm70fzdcvNQfPoy49IaanKkTlRWF2JMzqhcJbhSFRZv7nPTvZJdcY7301IPmvW+/p0RgIWnQDLJxifsQ7g==" + } + } + }, + "caniuse-api": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/caniuse-api/-/caniuse-api-3.0.0.tgz", + "integrity": "sha512-bsTwuIg/BZZK/vreVTYYbSWoe2F+71P7K5QGEX+pT250DZbfU1MQ5prOKpPR+LL6uWKK3KMwMCAS74QB3Um1uw==", + "requires": { + "browserslist": "^4.0.0", + "caniuse-lite": "^1.0.0", + "lodash.memoize": "^4.1.2", + "lodash.uniq": "^4.5.0" + } + }, + "caniuse-lite": { + "version": "1.0.30001339", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001339.tgz", + "integrity": "sha512-Es8PiVqCe+uXdms0Gu5xP5PF2bxLR7OBp3wUzUnuO7OHzhOfCyg3hdiGWVPVxhiuniOzng+hTc1u3fEQ0TlkSQ==" + }, + "capital-case": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/capital-case/-/capital-case-1.0.4.tgz", + "integrity": "sha512-ds37W8CytHgwnhGGTi88pcPyR15qoNkOpYwmMMfnWqqWgESapLqvDx6huFjQ5vqWSn2Z06173XNA7LtMOeUh1A==", + "requires": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case-first": "^2.0.2" + } + }, + "caseless": { + "version": "0.12.0", + "resolved": "https://registry.npmmirror.com/caseless/-/caseless-0.12.0.tgz", + "integrity": "sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==" + }, + "ccount": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/ccount/-/ccount-1.1.0.tgz", + "integrity": "sha512-vlNK021QdI7PNeiUh/lKkC/mNHHfV0m/Ad5JoI0TYtlBnJAslM/JIkm/tGC88bkLIwO6OQ5uV6ztS6kVAtCDlg==" + }, + "chalk": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", + "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "requires": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "dependencies": { + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "requires": { + "color-convert": "^2.0.1" + } + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==" + }, + "supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "requires": { + "has-flag": "^4.0.0" + } + } + } + }, + "change-case": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/change-case/-/change-case-4.1.2.tgz", + "integrity": "sha512-bSxY2ws9OtviILG1EiY5K7NNxkqg/JnRnFxLtKQ96JaviiIxi7djMrSd0ECT9AC+lttClmYwKw53BWpOMblo7A==", + "requires": { + "camel-case": "^4.1.2", + "capital-case": "^1.0.4", + "constant-case": "^3.0.4", + "dot-case": "^3.0.4", + "header-case": "^2.0.4", + "no-case": "^3.0.4", + "param-case": "^3.0.4", + "pascal-case": "^3.1.2", + "path-case": "^3.0.4", + "sentence-case": "^3.0.4", + "snake-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "change-case-all": { + "version": "1.0.14", + "resolved": "https://registry.npmjs.org/change-case-all/-/change-case-all-1.0.14.tgz", + "integrity": "sha512-CWVm2uT7dmSHdO/z1CXT/n47mWonyypzBbuCy5tN7uMg22BsfkhwT6oHmFCAk+gL1LOOxhdbB9SZz3J1KTY3gA==", + "requires": { + "change-case": "^4.1.2", + "is-lower-case": "^2.0.2", + "is-upper-case": "^2.0.2", + "lower-case": "^2.0.2", + "lower-case-first": "^2.0.2", + "sponge-case": "^1.0.1", + "swap-case": "^2.0.2", + "title-case": "^3.0.3", + "upper-case": "^2.0.2", + "upper-case-first": "^2.0.2" + } + }, + "character-entities": { + "version": "1.2.4", + "resolved": "https://registry.npmmirror.com/character-entities/-/character-entities-1.2.4.tgz", + "integrity": "sha512-iBMyeEHxfVnIakwOuDXpVkc54HijNgCyQB2w0VfGQThle6NXn50zU6V/u+LDhxHcDUPojn6Kpga3PTAD8W1bQw==" + }, + "character-entities-html4": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-entities-html4/-/character-entities-html4-1.1.4.tgz", + "integrity": "sha512-HRcDxZuZqMx3/a+qrzxdBKBPUpxWEq9xw2OPZ3a/174ihfrQKVsFhqtthBInFy1zZ9GgZyFXOatNujm8M+El3g==" + }, + "character-entities-legacy": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-entities-legacy/-/character-entities-legacy-1.1.4.tgz", + "integrity": "sha512-3Xnr+7ZFS1uxeiUDvV02wQ+QDbc55o97tIV5zHScSPJpcLm/r0DFPcoY3tYRp+VZukxuMeKgXYmsXQHO05zQeA==" + }, + "character-reference-invalid": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/character-reference-invalid/-/character-reference-invalid-1.1.4.tgz", + "integrity": "sha512-mKKUkUbhPpQlCOfIuZkvSEgktjPFIsZKRRbC6KWVEMvlzblj3i3asQv5ODsrwt0N3pHAEvjP8KTQPHkp0+6jOg==" + }, + "chardet": { + "version": "0.7.0", + "resolved": "https://registry.npmjs.org/chardet/-/chardet-0.7.0.tgz", + "integrity": "sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA==" + }, + "cheerio": { + "version": "1.0.0-rc.10", + "resolved": "https://registry.npmjs.org/cheerio/-/cheerio-1.0.0-rc.10.tgz", + "integrity": "sha512-g0J0q/O6mW8z5zxQ3A8E8J1hUgp4SMOvEoW/x84OwyHKe/Zccz83PVT4y5Crcr530FV6NgmKI1qvGTKVl9XXVw==", + "requires": { + "cheerio-select": "^1.5.0", + "dom-serializer": "^1.3.2", + "domhandler": "^4.2.0", + "htmlparser2": "^6.1.0", + "parse5": "^6.0.1", + "parse5-htmlparser2-tree-adapter": "^6.0.1", + "tslib": "^2.2.0" + } + }, + "cheerio-select": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/cheerio-select/-/cheerio-select-1.6.0.tgz", + "integrity": "sha512-eq0GdBvxVFbqWgmCm7M3XGs1I8oLy/nExUnh6oLqmBditPO9AqQJrkslDpMun/hZ0yyTs8L0m85OHp4ho6Qm9g==", + "requires": { + "css-select": "^4.3.0", + "css-what": "^6.0.1", + "domelementtype": "^2.2.0", + "domhandler": "^4.3.1", + "domutils": "^2.8.0" + } + }, + "chokidar": { + "version": "3.5.3", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", + "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "requires": { + "anymatch": "~3.1.2", + "braces": "~3.0.2", + "fsevents": "~2.3.2", + "glob-parent": "~5.1.2", + "is-binary-path": "~2.1.0", + "is-glob": "~4.0.1", + "normalize-path": "~3.0.0", + "readdirp": "~3.6.0" + } + }, + "chownr": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/chownr/-/chownr-1.1.4.tgz", + "integrity": "sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg==" + }, + "chrome-trace-event": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz", + "integrity": "sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg==" + }, + "ci-info": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-2.0.0.tgz", + "integrity": "sha512-5tK7EtrZ0N+OLFMthtqOj4fI2Jeb88C4CAZPu25LDVUgXJ0A3Js4PMGqrn0JU1W0Mh1/Z8wZzYPxqUrXeBboCQ==" + }, + "cli-boxes": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/cli-boxes/-/cli-boxes-2.2.1.tgz", + "integrity": "sha512-y4coMcylgSCdVinjiDBuR8PCC2bLjyGTwEmPb9NHR/QaNU6EUOXcTY/s6VjGMD6ENSEaeQYHCY0GNGS5jfMwPw==" + }, + "cli-cursor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/cli-cursor/-/cli-cursor-3.1.0.tgz", + "integrity": "sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw==", + "requires": { + "restore-cursor": "^3.1.0" + } + }, + "cli-width": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/cli-width/-/cli-width-3.0.0.tgz", + "integrity": "sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw==" + }, + "clipboardy": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/clipboardy/-/clipboardy-2.3.0.tgz", + "integrity": "sha512-mKhiIL2DrQIsuXMgBgnfEHOZOryC7kY7YO//TN6c63wlEm3NG5tz+YgY5rVi29KCmq/QQjKYvM7a19+MDOTHOQ==", + "requires": { + "arch": "^2.1.1", + "execa": "^1.0.0", + "is-wsl": "^2.1.1" + }, + "dependencies": { + "execa": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/execa/-/execa-1.0.0.tgz", + "integrity": "sha512-adbxcyWV46qiHyvSp50TKt05tB4tK3HcmF7/nxfAdhnox83seTDbwnaqKO4sXRy7roHAIFqJP/Rw/AuEbX61LA==", + "requires": { + "cross-spawn": "^6.0.0", + "get-stream": "^4.0.0", + "is-stream": "^1.1.0", + "npm-run-path": "^2.0.0", + "p-finally": "^1.0.0", + "signal-exit": "^3.0.0", + "strip-eof": "^1.0.0" + } + }, + "get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "requires": { + "pump": "^3.0.0" + } + }, + "is-stream": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-1.1.0.tgz", + "integrity": "sha1-EtSj3U5o4Lec6428hBc66A2RykQ=" + }, + "npm-run-path": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-2.0.2.tgz", + "integrity": "sha1-NakjLfo11wZ7TLLd8jV7GHFTbF8=", + "requires": { + "path-key": "^2.0.0" + } + } + } + }, + "cliui": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-6.0.0.tgz", + "integrity": "sha512-t6wbgtoCXvAzst7QgXxJYqPt0usEfbgQdftEPbLL/cvv6HPE5VgvqCuAIDR0NgU52ds6rFwqrgakNLrHEjCbrQ==", + "requires": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.0", + "wrap-ansi": "^6.2.0" + } + }, + "clone": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/clone/-/clone-2.1.2.tgz", + "integrity": "sha1-G39Ln1kfHo+DZwQBYANFoCiHQ18=" + }, + "clone-deep": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/clone-deep/-/clone-deep-4.0.1.tgz", + "integrity": "sha512-neHB9xuzh/wk0dIHweyAXv2aPGZIVk3pLMe+/RNzINf17fe0OG96QroktYAUm7SM1PBnzTabaLboqqxDyMU+SQ==", + "requires": { + "is-plain-object": "^2.0.4", + "kind-of": "^6.0.2", + "shallow-clone": "^3.0.0" + } + }, + "clone-response": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/clone-response/-/clone-response-1.0.2.tgz", + "integrity": "sha1-0dyXOSAxTfZ/vrlCI7TuNQI56Ws=", + "requires": { + "mimic-response": "^1.0.0" + } + }, + "clsx": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/clsx/-/clsx-1.1.1.tgz", + "integrity": "sha512-6/bPho624p3S2pMyvP5kKBPXnI3ufHLObBFCfgx+LkeR5lg2XYy2hqZqUf45ypD8COn2bhgGJSUE+l5dhNBieA==" + }, + "coa": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/coa/-/coa-2.0.2.tgz", + "integrity": "sha512-q5/jG+YQnSy4nRTV4F7lPepBJZ8qBNJJDBuJdoejDyLXgmL7IEo+Le2JDZudFTFt7mrCqIRaSjws4ygRCTCAXA==", + "requires": { + "@types/q": "^1.5.1", + "chalk": "^2.4.1", + "q": "^1.1.2" + }, + "dependencies": { + "chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + } + } + }, + "code-point-at": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/code-point-at/-/code-point-at-1.1.0.tgz", + "integrity": "sha1-DQcLTQQ6W+ozovGkDi7bPZpMz3c=" + }, + "color": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/color/-/color-4.2.3.tgz", + "integrity": "sha512-1rXeuUUiGGrykh+CeBdu5Ie7OJwinCgQY0bc7GCRxy5xVHy+moaqkpL/jqQq0MtQOeYcrqEz4abc5f0KtU7W4A==", + "requires": { + "color-convert": "^2.0.1", + "color-string": "^1.9.0" + }, + "dependencies": { + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + } + } + }, + "color-convert": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.3.tgz", + "integrity": "sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==", + "requires": { + "color-name": "1.1.3" + } + }, + "color-name": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.3.tgz", + "integrity": "sha1-p9BVi9icQveV3UIyj3QIMcpTvCU=" + }, + "color-string": { + "version": "1.9.1", + "resolved": "https://registry.npmjs.org/color-string/-/color-string-1.9.1.tgz", + "integrity": "sha512-shrVawQFojnZv6xM40anx4CkoDP+fZsw/ZerEMsW/pyzsRbElpsL/DBVW7q3ExxwusdNXI3lXpuhEZkzs8p5Eg==", + "requires": { + "color-name": "^1.0.0", + "simple-swizzle": "^0.2.2" + } + }, + "colord": { + "version": "2.9.2", + "resolved": "https://registry.npmjs.org/colord/-/colord-2.9.2.tgz", + "integrity": "sha512-Uqbg+J445nc1TKn4FoDPS6ZZqAvEDnwrH42yo8B40JSOgSLxMZ/gt3h4nmCtPLQeXhjJJkqBx7SCY35WnIixaQ==" + }, + "colorette": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/colorette/-/colorette-1.4.0.tgz", + "integrity": "sha512-Y2oEozpomLn7Q3HFP7dpww7AtMJplbM9lGZP6RDfHqmbeRjiwRg4n6VM6j4KLmRke85uWEI7JqF17f3pqdRA0g==" + }, + "combined-stream": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz", + "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==", + "requires": { + "delayed-stream": "~1.0.0" + } + }, + "comma-separated-tokens": { + "version": "1.0.8", + "resolved": "https://registry.npmmirror.com/comma-separated-tokens/-/comma-separated-tokens-1.0.8.tgz", + "integrity": "sha512-GHuDRO12Sypu2cV70d1dkA2EUmXHgntrzbpvOB+Qy+49ypNfGgFQIC2fhhXbnyrJRynDCAARsT7Ou0M6hirpfw==" + }, + "command-exists": { + "version": "1.2.9", + "resolved": "https://registry.npmjs.org/command-exists/-/command-exists-1.2.9.tgz", + "integrity": "sha512-LTQ/SGc+s0Xc0Fu5WaKnR0YiygZkm9eKFvyS+fRsU7/ZWFF8ykFM6Pc9aCVf1+xasOOZpO3BAVgVrKvsqKHV7w==" + }, + "commander": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/commander/-/commander-7.2.0.tgz", + "integrity": "sha512-QrWXB+ZQSVPmIWIhtEO9H+gwHaMGYiF5ChvoJ+K9ZGHG/sVsa6yiesAD1GC/x46sET00Xlwo1u49RVVVzvcSkw==" + }, + "common-tags": { + "version": "1.8.2", + "resolved": "https://registry.npmjs.org/common-tags/-/common-tags-1.8.2.tgz", + "integrity": "sha512-gk/Z852D2Wtb//0I+kRFNKKE9dIIVirjoqPoA1wJU+XePVXZfGeBpk45+A1rKO4Q43prqWBNY/MiIeRLbPWUaA==" + }, + "commondir": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/commondir/-/commondir-1.0.1.tgz", + "integrity": "sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs=" + }, + "component-emitter": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.3.0.tgz", + "integrity": "sha512-Rd3se6QB+sO1TwqZjscQrurpEPIfO0/yYnSin6Q/rD3mOutHvUrCAhJub3r90uNb+SESBuE0QYoB90YdfatsRg==" + }, + "compose-function": { + "version": "3.0.3", + "resolved": "https://registry.npmmirror.com/compose-function/-/compose-function-3.0.3.tgz", + "integrity": "sha512-xzhzTJ5eC+gmIzvZq+C3kCJHsp9os6tJkrigDRZclyGtOKINbZtE8n1Tzmeh32jW+BUDPbvZpibwvJHBLGMVwg==", + "requires": { + "arity-n": "^1.0.4" + } + }, + "compress-brotli": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/compress-brotli/-/compress-brotli-1.3.8.tgz", + "integrity": "sha512-lVcQsjhxhIXsuupfy9fmZUFtAIdBmXA7EGY6GBdgZ++qkM9zG4YFT8iU7FoBxzryNDMOpD1HIFHUSX4D87oqhQ==", + "requires": { + "@types/json-buffer": "~3.0.0", + "json-buffer": "~3.0.1" + } + }, + "compressible": { + "version": "2.0.18", + "resolved": "https://registry.npmjs.org/compressible/-/compressible-2.0.18.tgz", + "integrity": "sha512-AF3r7P5dWxL8MxyITRMlORQNaOA2IkAFaTr4k7BUumjPtRpGDTZpl0Pb1XCO6JeDCBdp126Cgs9sMxqSjgYyRg==", + "requires": { + "mime-db": ">= 1.43.0 < 2" + } + }, + "compression": { + "version": "1.7.4", + "resolved": "https://registry.npmjs.org/compression/-/compression-1.7.4.tgz", + "integrity": "sha512-jaSIDzP9pZVS4ZfQ+TzvtiWhdpFhE2RDHz8QJkpX9SIpLq88VueF5jJw6t+6CUQcAoA6t+x89MLrWAqpfDE8iQ==", + "requires": { + "accepts": "~1.3.5", + "bytes": "3.0.0", + "compressible": "~2.0.16", + "debug": "2.6.9", + "on-headers": "~1.0.2", + "safe-buffer": "5.1.2", + "vary": "~1.1.2" + }, + "dependencies": { + "bytes": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/bytes/-/bytes-3.0.0.tgz", + "integrity": "sha1-0ygVQE1olpn4Wk6k+odV3ROpYEg=" + }, + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=" + }, + "concat-stream": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/concat-stream/-/concat-stream-1.6.2.tgz", + "integrity": "sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw==", + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "requires": { + "safe-buffer": "~5.1.0" + } + } + } + }, + "configstore": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/configstore/-/configstore-5.0.1.tgz", + "integrity": "sha512-aMKprgk5YhBNyH25hj8wGt2+D52Sw1DRRIzqBwLp2Ya9mFmY8KPvvtvmna8SxVR9JMZ4kzMD68N22vlaRpkeFA==", + "requires": { + "dot-prop": "^5.2.0", + "graceful-fs": "^4.1.2", + "make-dir": "^3.0.0", + "unique-string": "^2.0.0", + "write-file-atomic": "^3.0.0", + "xdg-basedir": "^4.0.0" + } + }, + "confusing-browser-globals": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/confusing-browser-globals/-/confusing-browser-globals-1.0.11.tgz", + "integrity": "sha512-JsPKdmh8ZkmnHxDk55FZ1TqVLvEQTvoByJZRN9jzI0UjxK/QgAmsphz7PGtqgPieQZ/CQcHWXCR7ATDNhGe+YA==" + }, + "console-control-strings": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz", + "integrity": "sha1-PXz0Rk22RG6mRL9LOVB/mFEAjo4=" + }, + "constant-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/constant-case/-/constant-case-3.0.4.tgz", + "integrity": "sha512-I2hSBi7Vvs7BEuJDr5dDHfzb/Ruj3FyvFyh7KLilAjNQw3Be+xgqUBA2W6scVEcL0hL1dwPRtIqEPVUCKkSsyQ==", + "requires": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case": "^2.0.2" + } + }, + "content-disposition": { + "version": "0.5.4", + "resolved": "https://registry.npmjs.org/content-disposition/-/content-disposition-0.5.4.tgz", + "integrity": "sha512-FveZTNuGw04cxlAiWbzi6zTAL/lhehaWbTtgluJh4/E95DqMwTmha3KZN1aAWA8cFIhHzMZUvLevkw5Rqk+tSQ==", + "requires": { + "safe-buffer": "5.2.1" + }, + "dependencies": { + "safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + } + } + }, + "content-type": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/content-type/-/content-type-1.0.4.tgz", + "integrity": "sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA==" + }, + "convert-hrtime": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/convert-hrtime/-/convert-hrtime-3.0.0.tgz", + "integrity": "sha512-7V+KqSvMiHp8yWDuwfww06XleMWVVB9b9tURBx+G7UTADuo5hYPuowKloz4OzOqbPezxgo+fdQ1522WzPG4OeA==" + }, + "convert-source-map": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-1.8.0.tgz", + "integrity": "sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA==", + "requires": { + "safe-buffer": "~5.1.1" + } + }, + "cookie": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.4.2.tgz", + "integrity": "sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA==" + }, + "cookie-signature": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/cookie-signature/-/cookie-signature-1.0.6.tgz", + "integrity": "sha1-4wOogrNCzD7oylE6eZmXNNqzriw=" + }, + "core-js": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-3.22.5.tgz", + "integrity": "sha512-VP/xYuvJ0MJWRAobcmQ8F2H6Bsn+s7zqAAjFaHGBMc5AQm7zaelhD1LGduFn2EehEcQcU+br6t+fwbpQ5d1ZWA==" + }, + "core-js-compat": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js-compat/-/core-js-compat-3.22.5.tgz", + "integrity": "sha512-rEF75n3QtInrYICvJjrAgV03HwKiYvtKHdPtaba1KucG+cNZ4NJnH9isqt979e67KZlhpbCOTwnsvnIr+CVeOg==", + "requires": { + "browserslist": "^4.20.3", + "semver": "7.0.0" + }, + "dependencies": { + "semver": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.0.0.tgz", + "integrity": "sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A==" + } + } + }, + "core-js-pure": { + "version": "3.22.5", + "resolved": "https://registry.npmjs.org/core-js-pure/-/core-js-pure-3.22.5.tgz", + "integrity": "sha512-8xo9R00iYD7TcV7OrC98GwxiUEAabVWO3dix+uyWjnYrx9fyASLlIX+f/3p5dW5qByaP2bcZ8X/T47s55et/tA==" + }, + "core-util-is": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.3.tgz", + "integrity": "sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ==" + }, + "cors": { + "version": "2.8.5", + "resolved": "https://registry.npmjs.org/cors/-/cors-2.8.5.tgz", + "integrity": "sha512-KIHbLJqu73RGr/hnbrO9uBeixNGuvSQjul/jdFvS/KFSIH1hWVd1ng7zOHx+YrEfInLG7q4n6GHQ9cDtxv/P6g==", + "requires": { + "object-assign": "^4", + "vary": "^1" + } + }, + "cosmiconfig": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.1.tgz", + "integrity": "sha512-a1YWNUV2HwGimB7dU2s1wUMurNKjpx60HxBB6xUM8Re+2s1g1IIfJvFR0/iCF+XHdE0GMTKTuLR32UQff4TEyQ==", + "requires": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.2.1", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.10.0" + } + }, + "cosmiconfig-toml-loader": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig-toml-loader/-/cosmiconfig-toml-loader-1.0.0.tgz", + "integrity": "sha512-H/2gurFWVi7xXvCyvsWRLCMekl4tITJcX0QEsDMpzxtuxDyM59xLatYNg4s/k9AA/HdtCYfj2su8mgA0GSDLDA==", + "requires": { + "@iarna/toml": "^2.2.5" + } + }, + "create-gatsby": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/create-gatsby/-/create-gatsby-2.14.0.tgz", + "integrity": "sha512-Q92Omw5zPTKRrv5XDcsIVzBqSIHwl3T1lpOjQhSrQd42LDKUFAuE8zf/kTWT0QXo9cacBC+diUWIRxkqIZVKzQ==", + "requires": { + "@babel/runtime": "^7.15.4" + } + }, + "create-require": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/create-require/-/create-require-1.1.1.tgz", + "integrity": "sha512-dcKFX3jn0MpIaXjisoRvexIJVEKzaq7z2rZKxf+MSr9TkdmHmsU4m2lcLojrj/FHl8mk5VxMmYA+ftRkP/3oKQ==" + }, + "cross-fetch": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/cross-fetch/-/cross-fetch-3.1.5.tgz", + "integrity": "sha512-lvb1SBsI0Z7GDwmuid+mU3kWVBwTVUbe7S0H52yaaAdQOXq2YktTCZdlAcNKFzE6QtRz0snpw9bNiPeOIkkQvw==", + "requires": { + "node-fetch": "2.6.7" + } + }, + "cross-spawn": { + "version": "6.0.5", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-6.0.5.tgz", + "integrity": "sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ==", + "requires": { + "nice-try": "^1.0.4", + "path-key": "^2.0.1", + "semver": "^5.5.0", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + }, + "dependencies": { + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + } + } + }, + "crypto-random-string": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/crypto-random-string/-/crypto-random-string-2.0.0.tgz", + "integrity": "sha512-v1plID3y9r/lPhviJ1wrXpLeyUIGAZ2SHNYTEapm7/8A9nLPoyvVp3RK/EPFqn5kEznyWgYZNsRtYYIWbuG8KA==" + }, + "css": { + "version": "2.2.4", + "resolved": "https://registry.npmmirror.com/css/-/css-2.2.4.tgz", + "integrity": "sha512-oUnjmWpy0niI3x/mPL8dVEI1l7MnG3+HHyRPHf+YFSbK+svOhXpmSOcDURUh2aOCgl2grzrOPt1nHLuCVFULLw==", + "requires": { + "inherits": "^2.0.3", + "source-map": "^0.6.1", + "source-map-resolve": "^0.5.2", + "urix": "^0.1.0" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "css-declaration-sorter": { + "version": "6.2.2", + "resolved": "https://registry.npmjs.org/css-declaration-sorter/-/css-declaration-sorter-6.2.2.tgz", + "integrity": "sha512-Ufadglr88ZLsrvS11gjeu/40Lw74D9Am/Jpr3LlYm5Q4ZP5KdlUhG+6u2EjyXeZcxmZ2h1ebCKngDjolpeLHpg==", + "requires": {} + }, + "css-loader": { + "version": "5.2.7", + "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-5.2.7.tgz", + "integrity": "sha512-Q7mOvpBNBG7YrVGMxRxcBJZFL75o+cH2abNASdibkj/fffYD8qWbInZrD0S9ccI6vZclF3DsHE7njGlLtaHbhg==", + "requires": { + "icss-utils": "^5.1.0", + "loader-utils": "^2.0.0", + "postcss": "^8.2.15", + "postcss-modules-extract-imports": "^3.0.0", + "postcss-modules-local-by-default": "^4.0.0", + "postcss-modules-scope": "^3.0.0", + "postcss-modules-values": "^4.0.0", + "postcss-value-parser": "^4.1.0", + "schema-utils": "^3.0.0", + "semver": "^7.3.5" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "css-minimizer-webpack-plugin": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/css-minimizer-webpack-plugin/-/css-minimizer-webpack-plugin-2.0.0.tgz", + "integrity": "sha512-cG/uc94727tx5pBNtb1Sd7gvUPzwmcQi1lkpfqTpdkuNq75hJCw7bIVsCNijLm4dhDcr1atvuysl2rZqOG8Txw==", + "requires": { + "cssnano": "^5.0.0", + "jest-worker": "^26.3.0", + "p-limit": "^3.0.2", + "postcss": "^8.2.9", + "schema-utils": "^3.0.0", + "serialize-javascript": "^5.0.1", + "source-map": "^0.6.1" + }, + "dependencies": { + "p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "requires": { + "yocto-queue": "^0.1.0" + } + }, + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "css-select": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/css-select/-/css-select-4.3.0.tgz", + "integrity": "sha512-wPpOYtnsVontu2mODhA19JrqWxNsfdatRKd64kmpRbQgh1KtItko5sTnEpPdpSaJszTOhEMlF/RPz28qj4HqhQ==", + "requires": { + "boolbase": "^1.0.0", + "css-what": "^6.0.1", + "domhandler": "^4.3.1", + "domutils": "^2.8.0", + "nth-check": "^2.0.1" + } + }, + "css-select-base-adapter": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/css-select-base-adapter/-/css-select-base-adapter-0.1.1.tgz", + "integrity": "sha512-jQVeeRG70QI08vSTwf1jHxp74JoZsr2XSgETae8/xC8ovSnL2WF87GTLO86Sbwdt2lK4Umg4HnnwMO4YF3Ce7w==" + }, + "css-selector-parser": { + "version": "1.4.1", + "resolved": "https://registry.npmmirror.com/css-selector-parser/-/css-selector-parser-1.4.1.tgz", + "integrity": "sha512-HYPSb7y/Z7BNDCOrakL4raGO2zltZkbeXyAd6Tg9obzix6QhzxCotdBl6VT0Dv4vZfJGVz3WL/xaEI9Ly3ul0g==" + }, + "css-tree": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.1.3.tgz", + "integrity": "sha512-tRpdppF7TRazZrjJ6v3stzv93qxRcSsFmW6cX0Zm2NVKpxE1WV1HblnghVv9TreireHkqI/VDEsfolRF1p6y7Q==", + "requires": { + "mdn-data": "2.0.14", + "source-map": "^0.6.1" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "css-what": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/css-what/-/css-what-6.1.0.tgz", + "integrity": "sha512-HTUrgRJ7r4dsZKU6GjmpfRK1O76h97Z8MfS1G0FozR+oF2kG6Vfe8JE6zwrkbxigziPHinCJ+gCPjA9EaBDtRw==" + }, + "css.escape": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/css.escape/-/css.escape-1.5.1.tgz", + "integrity": "sha1-QuJ9T6BK4y+TGktNQZH6nN3ul8s=" + }, + "cssesc": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-3.0.0.tgz", + "integrity": "sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg==" + }, + "cssfilter": { + "version": "0.0.10", + "resolved": "https://registry.npmjs.org/cssfilter/-/cssfilter-0.0.10.tgz", + "integrity": "sha1-xtJnJjKi5cg+AT5oZKQs6N79IK4=" + }, + "cssnano": { + "version": "5.1.7", + "resolved": "https://registry.npmjs.org/cssnano/-/cssnano-5.1.7.tgz", + "integrity": "sha512-pVsUV6LcTXif7lvKKW9ZrmX+rGRzxkEdJuVJcp5ftUjWITgwam5LMZOgaTvUrWPkcORBey6he7JKb4XAJvrpKg==", + "requires": { + "cssnano-preset-default": "^5.2.7", + "lilconfig": "^2.0.3", + "yaml": "^1.10.2" + } + }, + "cssnano-preset-default": { + "version": "5.2.7", + "resolved": "https://registry.npmjs.org/cssnano-preset-default/-/cssnano-preset-default-5.2.7.tgz", + "integrity": "sha512-JiKP38ymZQK+zVKevphPzNSGHSlTI+AOwlasoSRtSVMUU285O7/6uZyd5NbW92ZHp41m0sSHe6JoZosakj63uA==", + "requires": { + "css-declaration-sorter": "^6.2.2", + "cssnano-utils": "^3.1.0", + "postcss-calc": "^8.2.3", + "postcss-colormin": "^5.3.0", + "postcss-convert-values": "^5.1.0", + "postcss-discard-comments": "^5.1.1", + "postcss-discard-duplicates": "^5.1.0", + "postcss-discard-empty": "^5.1.1", + "postcss-discard-overridden": "^5.1.0", + "postcss-merge-longhand": "^5.1.4", + "postcss-merge-rules": "^5.1.1", + "postcss-minify-font-values": "^5.1.0", + "postcss-minify-gradients": "^5.1.1", + "postcss-minify-params": "^5.1.2", + "postcss-minify-selectors": "^5.2.0", + "postcss-normalize-charset": "^5.1.0", + "postcss-normalize-display-values": "^5.1.0", + "postcss-normalize-positions": "^5.1.0", + "postcss-normalize-repeat-style": "^5.1.0", + "postcss-normalize-string": "^5.1.0", + "postcss-normalize-timing-functions": "^5.1.0", + "postcss-normalize-unicode": "^5.1.0", + "postcss-normalize-url": "^5.1.0", + "postcss-normalize-whitespace": "^5.1.1", + "postcss-ordered-values": "^5.1.1", + "postcss-reduce-initial": "^5.1.0", + "postcss-reduce-transforms": "^5.1.0", + "postcss-svgo": "^5.1.0", + "postcss-unique-selectors": "^5.1.1" + } + }, + "cssnano-utils": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/cssnano-utils/-/cssnano-utils-3.1.0.tgz", + "integrity": "sha512-JQNR19/YZhz4psLX/rQ9M83e3z2Wf/HdJbryzte4a3NSuafyp9w/I4U+hx5C2S9g41qlstH7DEWnZaaj83OuEA==", + "requires": {} + }, + "csso": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/csso/-/csso-4.2.0.tgz", + "integrity": "sha512-wvlcdIbf6pwKEk7vHj8/Bkc0B4ylXZruLvOgs9doS5eOsOpuodOV2zJChSpkp+pRpYQLQMeF04nr3Z68Sta9jA==", + "requires": { + "css-tree": "^1.1.2" + } + }, + "csstype": { + "version": "3.0.11", + "resolved": "https://registry.npmjs.org/csstype/-/csstype-3.0.11.tgz", + "integrity": "sha512-sa6P2wJ+CAbgyy4KFssIb/JNMLxFvKF1pCYCSXS8ZMuqZnMsrxqI2E5sPyoTpxoPU/gVZMzr2zjOfg8GIZOMsw==" + }, + "d": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/d/-/d-1.0.1.tgz", + "integrity": "sha512-m62ShEObQ39CfralilEQRjH6oAMtNCV1xJyEx5LpRYUVN+EviphDgUc/F3hnYbADmkiNs67Y+3ylmlG7Lnu+FA==", + "requires": { + "es5-ext": "^0.10.50", + "type": "^1.0.1" + } + }, + "damerau-levenshtein": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/damerau-levenshtein/-/damerau-levenshtein-1.0.8.tgz", + "integrity": "sha512-sdQSFB7+llfUcQHUQO3+B8ERRj0Oa4w9POWMI/puGtuf7gFywGmkaLCElnudfTiKZV+NvHqL0ifzdrI8Ro7ESA==" + }, + "dashdash": { + "version": "1.14.1", + "resolved": "https://registry.npmmirror.com/dashdash/-/dashdash-1.14.1.tgz", + "integrity": "sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==", + "requires": { + "assert-plus": "^1.0.0" + } + }, + "dataloader": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/dataloader/-/dataloader-2.0.0.tgz", + "integrity": "sha512-YzhyDAwA4TaQIhM5go+vCLmU0UikghC/t9DTQYZR2M/UvZ1MdOhPezSDZcjj9uqQJOMqjLcpWtyW2iNINdlatQ==" + }, + "date-fns": { + "version": "2.28.0", + "resolved": "https://registry.npmjs.org/date-fns/-/date-fns-2.28.0.tgz", + "integrity": "sha512-8d35hViGYx/QH0icHYCeLmsLmMUheMmTyV9Fcm6gvNwdw31yXXH+O85sOBJ+OLnLQMKZowvpKb6FgMIQjcpvQw==" + }, + "debug": { + "version": "3.2.7", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", + "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", + "requires": { + "ms": "^2.1.1" + } + }, + "decamelize": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-1.2.0.tgz", + "integrity": "sha1-9lNNFRSCabIDUue+4m9QH5oZEpA=" + }, + "decamelize-keys": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/decamelize-keys/-/decamelize-keys-1.1.0.tgz", + "integrity": "sha512-ocLWuYzRPoS9bfiSdDd3cxvrzovVMZnRDVEzAs+hWIVXGDbHxWMECij2OBuyB/An0FFW/nLuq6Kv1i/YC5Qfzg==", + "requires": { + "decamelize": "^1.1.0", + "map-obj": "^1.0.0" + }, + "dependencies": { + "map-obj": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/map-obj/-/map-obj-1.0.1.tgz", + "integrity": "sha512-7N/q3lyZ+LVCp7PzuxrJr4KMbBE2hW7BT7YNia330OFxIf4d3r5zVpicP2650l7CPN6RM9zOJRl3NGpqSiw3Eg==" + } + } + }, + "decode-uri-component": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/decode-uri-component/-/decode-uri-component-0.2.0.tgz", + "integrity": "sha1-6zkTMzRYd1y4TNGh+uBiEGu4dUU=" + }, + "decompress-response": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-6.0.0.tgz", + "integrity": "sha512-aW35yZM6Bb/4oJlZncMH2LCoZtJXTRxES17vE3hoRiowU2kWHaJKFkSBDnDR+cm9J+9QhXmREyIfv0pji9ejCQ==", + "requires": { + "mimic-response": "^3.1.0" + }, + "dependencies": { + "mimic-response": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/mimic-response/-/mimic-response-3.1.0.tgz", + "integrity": "sha512-z0yWI+4FDrrweS8Zmt4Ej5HdJmky15+L2e6Wgn3+iK5fWzb6T3fhNFq2+MeTRb064c6Wr4N/wv0DzQTjNzHNGQ==" + } + } + }, + "deep-extend": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/deep-extend/-/deep-extend-0.6.0.tgz", + "integrity": "sha512-LOHxIOaPYdHlJRtCQfDIVZtfw/ufM8+rVj649RIHzcm/vGwQRXFt6OPqIFWsm2XEMrNIEtWR64sY1LEKD2vAOA==" + }, + "deep-is": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", + "integrity": "sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==" + }, + "deepmerge": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/deepmerge/-/deepmerge-4.2.2.tgz", + "integrity": "sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg==" + }, + "defer-to-connect": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-2.0.1.tgz", + "integrity": "sha512-4tvttepXG1VaYGrRibk5EwJd1t4udunSOVMdLSAL6mId1ix438oPwPZMALY41FCijukO1L0twNcGsdzS7dHgDg==" + }, + "define-lazy-prop": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz", + "integrity": "sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og==" + }, + "define-properties": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.1.4.tgz", + "integrity": "sha512-uckOqKcfaVvtBdsVkdPv3XjveQJsNQqmhXgRi8uhvWWuPYZCNlzT8qAyblUgNoXdHdjMTzAqeGjAoli8f+bzPA==", + "requires": { + "has-property-descriptors": "^1.0.0", + "object-keys": "^1.1.1" + } + }, + "delayed-stream": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", + "integrity": "sha1-3zrhmayt+31ECqrgsp4icrJOxhk=" + }, + "delegates": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", + "integrity": "sha1-hMbhWbgZBP3KWaDvRM2HDTElD5o=" + }, + "depd": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/depd/-/depd-2.0.0.tgz", + "integrity": "sha512-g7nH6P6dyDioJogAAGprGpCtVImJhpPk/roCzdb3fIh61/s/nPsfR6onyMwkCAR/OlC3yBC0lESvUoQEAssIrw==" + }, + "dependency-graph": { + "version": "0.11.0", + "resolved": "https://registry.npmjs.org/dependency-graph/-/dependency-graph-0.11.0.tgz", + "integrity": "sha512-JeMq7fEshyepOWDfcfHK06N3MhyPhz++vtqWhMT5O9A3K42rdsEDpfdVqjaqaAhsw6a+ZqeDvQVtD0hFHQWrzg==" + }, + "destroy": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/destroy/-/destroy-1.2.0.tgz", + "integrity": "sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg==" + }, + "detect-libc": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-1.0.3.tgz", + "integrity": "sha1-+hN8S9aY7fVc1c0CrFWfkaTEups=" + }, + "detect-port": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/detect-port/-/detect-port-1.3.0.tgz", + "integrity": "sha512-E+B1gzkl2gqxt1IhUzwjrxBKRqx1UzC3WLONHinn8S3T6lwV/agVCyitiFOsGJ/eYuEUBvD71MZHy3Pv1G9doQ==", + "requires": { + "address": "^1.0.1", + "debug": "^2.6.0" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "devcert": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/devcert/-/devcert-1.2.1.tgz", + "integrity": "sha512-R7DqtMtsNmFVY75kzRHXON3hXoJili2xxlEcZgHi0VHSx8aJECfm7ZqAquXzTeAM/I9f8G2pHc/zq5k6iXHQzA==", + "requires": { + "@types/configstore": "^2.1.1", + "@types/debug": "^0.0.30", + "@types/get-port": "^3.2.0", + "@types/glob": "^5.0.34", + "@types/lodash": "^4.14.92", + "@types/mkdirp": "^0.5.2", + "@types/node": "^8.5.7", + "@types/rimraf": "^2.0.2", + "@types/tmp": "^0.0.33", + "application-config-path": "^0.1.0", + "command-exists": "^1.2.4", + "debug": "^3.1.0", + "eol": "^0.9.1", + "get-port": "^3.2.0", + "glob": "^7.1.2", + "is-valid-domain": "^0.1.6", + "lodash": "^4.17.4", + "mkdirp": "^0.5.1", + "password-prompt": "^1.0.4", + "rimraf": "^2.6.2", + "sudo-prompt": "^8.2.0", + "tmp": "^0.0.33", + "tslib": "^1.10.0" + }, + "dependencies": { + "@types/node": { + "version": "8.10.66", + "resolved": "https://registry.npmjs.org/@types/node/-/node-8.10.66.tgz", + "integrity": "sha512-tktOkFUA4kXx2hhhrB8bIFb5TbwzS4uOhKEmwiD+NoiL0qtP2OQ9mFldbgD4dV1djrlBYP6eBuQZiWjuHUpqFw==" + }, + "rimraf": { + "version": "2.7.1", + "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.7.1.tgz", + "integrity": "sha512-uWjbaKIK3T1OSVptzX7Nl6PvQ3qAGtKEtVRjRuazjfL3Bx5eI409VZSqgND+4UNnmzLVdPj9FqFJNPqBZFve4w==", + "requires": { + "glob": "^7.1.3" + } + }, + "tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==", + "requires": { + "os-tmpdir": "~1.0.2" + } + }, + "tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + } + } + }, + "dicer": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/dicer/-/dicer-0.2.5.tgz", + "integrity": "sha1-WZbAhrszIYyBLAkL3cCc0S+stw8=", + "requires": { + "readable-stream": "1.1.x", + "streamsearch": "0.1.2" + }, + "dependencies": { + "isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8=" + }, + "readable-stream": { + "version": "1.1.14", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", + "integrity": "sha1-fPTFTvZI44EwhMY23SB54WbAgdk=", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=" + } + } + }, + "diff": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/diff/-/diff-4.0.2.tgz", + "integrity": "sha512-58lmxKSA4BNyLz+HHMUzlOEpg09FV+ev6ZMe3vJihgdxzgcwZ8VoEEPmALCZG9LmqfVoNMMKpttIYTVG6uDY7A==" + }, + "dir-glob": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-3.0.1.tgz", + "integrity": "sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==", + "requires": { + "path-type": "^4.0.0" + } + }, + "doctrine": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", + "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", + "requires": { + "esutils": "^2.0.2" + } + }, + "dom-converter": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/dom-converter/-/dom-converter-0.2.0.tgz", + "integrity": "sha512-gd3ypIPfOMr9h5jIKq8E3sHOTCjeirnl0WK5ZdS1AW0Odt0b1PaWaHdJ4Qk4klv+YB9aJBS7mESXjFoDQPu6DA==", + "requires": { + "utila": "~0.4" + } + }, + "dom-serializer": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-1.4.1.tgz", + "integrity": "sha512-VHwB3KfrcOOkelEG2ZOfxqLZdfkil8PtJi4P8N2MMXucZq2yLp75ClViUlOVwyoHEDjYU433Aq+5zWP61+RGag==", + "requires": { + "domelementtype": "^2.0.1", + "domhandler": "^4.2.0", + "entities": "^2.0.0" + } + }, + "dom-walk": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/dom-walk/-/dom-walk-0.1.2.tgz", + "integrity": "sha512-6QvTW9mrGeIegrFXdtQi9pk7O/nSK6lSdXW2eqUspN5LWD7UTji2Fqw5V2YLjBpHEoU9Xl/eUWNpDeZvoyOv2w==" + }, + "domelementtype": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-2.3.0.tgz", + "integrity": "sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw==" + }, + "domhandler": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-4.3.1.tgz", + "integrity": "sha512-GrwoxYN+uWlzO8uhUXRl0P+kHE4GtVPfYzVLcUxPL7KNdHKj66vvlhiweIHqYYXWlw+T8iLMp42Lm67ghw4WMQ==", + "requires": { + "domelementtype": "^2.2.0" + } + }, + "domutils": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/domutils/-/domutils-2.8.0.tgz", + "integrity": "sha512-w96Cjofp72M5IIhpjgobBimYEfoPjx1Vx0BSX9P30WBdZW2WIKU0T1Bd0kz2eNZ9ikjKgHbEyKx8BB6H1L3h3A==", + "requires": { + "dom-serializer": "^1.0.1", + "domelementtype": "^2.2.0", + "domhandler": "^4.2.0" + } + }, + "dot-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/dot-case/-/dot-case-3.0.4.tgz", + "integrity": "sha512-Kv5nKlh6yRrdrGvxeJ2e5y2eRUpkUosIW4A2AS38zwSz27zu7ufDwQPi5Jhs3XAlGNetl3bmnGhQsMtkKJnj3w==", + "requires": { + "no-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "dot-prop": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/dot-prop/-/dot-prop-5.3.0.tgz", + "integrity": "sha512-QM8q3zDe58hqUqjraQOmzZ1LIH9SWQJTlEKCH4kJ2oQvLZk7RbQXvtDM2XEq3fwkV9CCvvH4LA0AV+ogFsBM2Q==", + "requires": { + "is-obj": "^2.0.0" + } + }, + "dotenv": { + "version": "8.6.0", + "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-8.6.0.tgz", + "integrity": "sha512-IrPdXQsk2BbzvCBGBOTmmSH5SodmqZNt4ERAZDmW4CT+tL8VtvinqywuANaFu4bOMWki16nqf0e4oC0QIaDr/g==" + }, + "dotenv-expand": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/dotenv-expand/-/dotenv-expand-5.1.0.tgz", + "integrity": "sha512-YXQl1DSa4/PQyRfgrv6aoNjhasp/p4qs9FjJ4q4cQk+8m4r6k4ZSiEyytKG8f8W9gi8WsQtIObNmKd+tMzNTmA==" + }, + "duplexer": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.2.tgz", + "integrity": "sha512-jtD6YG370ZCIi/9GTaJKQxWTZD045+4R4hTk/x1UyoqadyJ9x9CgSi1RlVDQF8U2sxLLSnFkCaMihqljHIWgMg==" + }, + "duplexer2": { + "version": "0.1.4", + "resolved": "https://registry.npmmirror.com/duplexer2/-/duplexer2-0.1.4.tgz", + "integrity": "sha512-asLFVfWWtJ90ZyOUHMqk7/S2w2guQKxUI2itj3d92ADHhxUSbCMGi1f1cBcJ7xM1To+pE/Khbwo1yuNbMEPKeA==", + "requires": { + "readable-stream": "^2.0.2" + }, + "dependencies": { + "readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "requires": { + "safe-buffer": "~5.1.0" + } + } + } + }, + "duplexer3": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/duplexer3/-/duplexer3-0.1.4.tgz", + "integrity": "sha1-7gHdHKwO08vH/b6jfcCo8c4ALOI=" + }, + "ecc-jsbn": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz", + "integrity": "sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==", + "requires": { + "jsbn": "~0.1.0", + "safer-buffer": "^2.1.0" + } + }, + "ee-first": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/ee-first/-/ee-first-1.1.1.tgz", + "integrity": "sha1-WQxhFWsK4vTwJVcyoViyZrxWsh0=" + }, + "electron-to-chromium": { + "version": "1.4.137", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.137.tgz", + "integrity": "sha512-0Rcpald12O11BUogJagX3HsCN3FE83DSqWjgXoHo5a72KUKMSfI39XBgJpgNNxS9fuGzytaFjE06kZkiVFy2qA==" + }, + "emoji-regex": { + "version": "8.0.0", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" + }, + "emojis-list": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/emojis-list/-/emojis-list-3.0.0.tgz", + "integrity": "sha512-/kyM18EfinwXZbno9FyUGeFh87KC8HRQBQGildHZbEuRyWFOmv1U10o9BBp8XVZDVNNuQKyIGIu5ZYAAXJ0V2Q==" + }, + "encodeurl": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-1.0.2.tgz", + "integrity": "sha1-rT/0yG7C0CkyL1oCw6mmBslbP1k=" + }, + "end-of-stream": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.4.tgz", + "integrity": "sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==", + "requires": { + "once": "^1.4.0" + } + }, + "engine.io": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/engine.io/-/engine.io-4.1.2.tgz", + "integrity": "sha512-t5z6zjXuVLhXDMiFJPYsPOWEER8B0tIsD3ETgw19S1yg9zryvUfY3Vhtk3Gf4sihw/bQGIqQ//gjvVlu+Ca0bQ==", + "requires": { + "accepts": "~1.3.4", + "base64id": "2.0.0", + "cookie": "~0.4.1", + "cors": "~2.8.5", + "debug": "~4.3.1", + "engine.io-parser": "~4.0.0", + "ws": "~7.4.2" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "engine.io-client": { + "version": "4.1.4", + "resolved": "https://registry.npmjs.org/engine.io-client/-/engine.io-client-4.1.4.tgz", + "integrity": "sha512-843fqAdKeUMFqKi1sSjnR11tJ4wi8sIefu6+JC1OzkkJBmjtc/gM/rZ53tJfu5Iae/3gApm5veoS+v+gtT0+Fg==", + "requires": { + "base64-arraybuffer": "0.1.4", + "component-emitter": "~1.3.0", + "debug": "~4.3.1", + "engine.io-parser": "~4.0.1", + "has-cors": "1.1.0", + "parseqs": "0.0.6", + "parseuri": "0.0.6", + "ws": "~7.4.2", + "xmlhttprequest-ssl": "~1.6.2", + "yeast": "0.1.2" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "engine.io-parser": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/engine.io-parser/-/engine.io-parser-4.0.3.tgz", + "integrity": "sha512-xEAAY0msNnESNPc00e19y5heTPX4y/TJ36gr8t1voOaNmTojP9b3oK3BbJLFufW2XFPQaaijpFewm2g2Um3uqA==", + "requires": { + "base64-arraybuffer": "0.1.4" + } + }, + "enhanced-resolve": { + "version": "5.9.3", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.9.3.tgz", + "integrity": "sha512-Bq9VSor+kjvW3f9/MiiR4eE3XYgOl7/rS8lnSxbRbF3kS0B2r+Y9w5krBWxZgDxASVZbdYrn5wT4j/Wb0J9qow==", + "requires": { + "graceful-fs": "^4.2.4", + "tapable": "^2.2.0" + } + }, + "enquirer": { + "version": "2.3.6", + "resolved": "https://registry.npmjs.org/enquirer/-/enquirer-2.3.6.tgz", + "integrity": "sha512-yjNnPr315/FjS4zIsUxYguYUPP2e1NK4d7E7ZOLiyYCcbFBiTMyID+2wvm2w6+pZ/odMA7cRkjhsPbltwBOrLg==", + "requires": { + "ansi-colors": "^4.1.1" + } + }, + "entities": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/entities/-/entities-2.2.0.tgz", + "integrity": "sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A==" + }, + "env-paths": { + "version": "2.2.1", + "resolved": "https://registry.npmmirror.com/env-paths/-/env-paths-2.2.1.tgz", + "integrity": "sha512-+h1lkLKhZMTYjog1VEpJNG7NZJWcuc2DDk/qsqSTRRCOXiLjeQ1d1/udrUGhqMxUgAlwKNZ0cf2uqan5GLuS2A==" + }, + "envinfo": { + "version": "7.8.1", + "resolved": "https://registry.npmjs.org/envinfo/-/envinfo-7.8.1.tgz", + "integrity": "sha512-/o+BXHmB7ocbHEAs6F2EnG0ogybVVUdkRunTT2glZU9XAaGmhqskrvKwqXuDfNjEO0LZKWdejEEpnq8aM0tOaw==" + }, + "eol": { + "version": "0.9.1", + "resolved": "https://registry.npmjs.org/eol/-/eol-0.9.1.tgz", + "integrity": "sha512-Ds/TEoZjwggRoz/Q2O7SE3i4Jm66mqTDfmdHdq/7DKVk3bro9Q8h6WdXKdPqFLMoqxrDK5SVRzHVPOS6uuGtrg==" + }, + "error-ex": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/error-ex/-/error-ex-1.3.2.tgz", + "integrity": "sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g==", + "requires": { + "is-arrayish": "^0.2.1" + } + }, + "error-stack-parser": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/error-stack-parser/-/error-stack-parser-2.0.7.tgz", + "integrity": "sha512-chLOW0ZGRf4s8raLrDxa5sdkvPec5YdvwbFnqJme4rk0rFajP8mPtrDL1+I+CwrQDCjswDA5sREX7jYQDQs9vA==", + "requires": { + "stackframe": "^1.1.1" + } + }, + "es-abstract": { + "version": "1.20.0", + "resolved": "https://registry.npmjs.org/es-abstract/-/es-abstract-1.20.0.tgz", + "integrity": "sha512-URbD8tgRthKD3YcC39vbvSDrX23upXnPcnGAjQfgxXF5ID75YcENawc9ZX/9iTP9ptUyfCLIxTTuMYoRfiOVKA==", + "requires": { + "call-bind": "^1.0.2", + "es-to-primitive": "^1.2.1", + "function-bind": "^1.1.1", + "function.prototype.name": "^1.1.5", + "get-intrinsic": "^1.1.1", + "get-symbol-description": "^1.0.0", + "has": "^1.0.3", + "has-property-descriptors": "^1.0.0", + "has-symbols": "^1.0.3", + "internal-slot": "^1.0.3", + "is-callable": "^1.2.4", + "is-negative-zero": "^2.0.2", + "is-regex": "^1.1.4", + "is-shared-array-buffer": "^1.0.2", + "is-string": "^1.0.7", + "is-weakref": "^1.0.2", + "object-inspect": "^1.12.0", + "object-keys": "^1.1.1", + "object.assign": "^4.1.2", + "regexp.prototype.flags": "^1.4.1", + "string.prototype.trimend": "^1.0.5", + "string.prototype.trimstart": "^1.0.5", + "unbox-primitive": "^1.0.2" + } + }, + "es-module-lexer": { + "version": "0.9.3", + "resolved": "https://registry.npmjs.org/es-module-lexer/-/es-module-lexer-0.9.3.tgz", + "integrity": "sha512-1HQ2M2sPtxwnvOvT1ZClHyQDiggdNjURWpY2we6aMKCQiUVxTmVs2UYPLIrD84sS+kMdUwfBSylbJPwNnBrnHQ==" + }, + "es-shim-unscopables": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/es-shim-unscopables/-/es-shim-unscopables-1.0.0.tgz", + "integrity": "sha512-Jm6GPcCdC30eMLbZ2x8z2WuRwAws3zTBBKuusffYVUrNj/GVSUAZ+xKMaUpfNDR5IbyNA5LJbaecoUVbmUcB1w==", + "requires": { + "has": "^1.0.3" + } + }, + "es-to-primitive": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/es-to-primitive/-/es-to-primitive-1.2.1.tgz", + "integrity": "sha512-QCOllgZJtaUo9miYBcLChTUaHNjJF3PYs1VidD7AwiEj1kYxKeQTctLAezAOH5ZKRH0g2IgPn6KwB4IT8iRpvA==", + "requires": { + "is-callable": "^1.1.4", + "is-date-object": "^1.0.1", + "is-symbol": "^1.0.2" + } + }, + "es5-ext": { + "version": "0.10.61", + "resolved": "https://registry.npmjs.org/es5-ext/-/es5-ext-0.10.61.tgz", + "integrity": "sha512-yFhIqQAzu2Ca2I4SE2Au3rxVfmohU9Y7wqGR+s7+H7krk26NXhIRAZDgqd6xqjCEFUomDEA3/Bo/7fKmIkW1kA==", + "requires": { + "es6-iterator": "^2.0.3", + "es6-symbol": "^3.1.3", + "next-tick": "^1.1.0" + } + }, + "es6-iterator": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/es6-iterator/-/es6-iterator-2.0.3.tgz", + "integrity": "sha1-p96IkUGgWpSwhUQDstCg+/qY87c=", + "requires": { + "d": "1", + "es5-ext": "^0.10.35", + "es6-symbol": "^3.1.1" + } + }, + "es6-symbol": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/es6-symbol/-/es6-symbol-3.1.3.tgz", + "integrity": "sha512-NJ6Yn3FuDinBaBRWl/q5X/s4koRHBrgKAu+yGI6JCBeiu3qrcbJhwT2GeR/EXVfylRk8dpQVJoLEFhK+Mu31NA==", + "requires": { + "d": "^1.0.1", + "ext": "^1.1.2" + } + }, + "es6-weak-map": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/es6-weak-map/-/es6-weak-map-2.0.3.tgz", + "integrity": "sha512-p5um32HOTO1kP+w7PRnB+5lQ43Z6muuMuIMffvDN8ZB4GcnjLBV6zGStpbASIMk4DCAvEaamhe2zhyCb/QXXsA==", + "requires": { + "d": "1", + "es5-ext": "^0.10.46", + "es6-iterator": "^2.0.3", + "es6-symbol": "^3.1.1" + } + }, + "escalade": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", + "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==" + }, + "escape-goat": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/escape-goat/-/escape-goat-2.1.1.tgz", + "integrity": "sha512-8/uIhbG12Csjy2JEW7D9pHbreaVaS/OpN3ycnyvElTdwM5n6GY6W6e2IPemfvGZeUMqZ9A/3GqIZMgKnBhAw/Q==" + }, + "escape-html": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/escape-html/-/escape-html-1.0.3.tgz", + "integrity": "sha1-Aljq5NPQwJdN4cFpGI7wBR0dGYg=" + }, + "escape-string-regexp": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz", + "integrity": "sha1-G2HAViGQqN/2rjuyzwIAyhMLhtQ=" + }, + "eslint": { + "version": "7.32.0", + "resolved": "https://registry.npmjs.org/eslint/-/eslint-7.32.0.tgz", + "integrity": "sha512-VHZ8gX+EDfz+97jGcgyGCyRia/dPOd6Xh9yPv8Bl1+SoaIwD+a/vlrOmGRUyOYu7MwUhc7CxqeaDZU13S4+EpA==", + "requires": { + "@babel/code-frame": "7.12.11", + "@eslint/eslintrc": "^0.4.3", + "@humanwhocodes/config-array": "^0.5.0", + "ajv": "^6.10.0", + "chalk": "^4.0.0", + "cross-spawn": "^7.0.2", + "debug": "^4.0.1", + "doctrine": "^3.0.0", + "enquirer": "^2.3.5", + "escape-string-regexp": "^4.0.0", + "eslint-scope": "^5.1.1", + "eslint-utils": "^2.1.0", + "eslint-visitor-keys": "^2.0.0", + "espree": "^7.3.1", + "esquery": "^1.4.0", + "esutils": "^2.0.2", + "fast-deep-equal": "^3.1.3", + "file-entry-cache": "^6.0.1", + "functional-red-black-tree": "^1.0.1", + "glob-parent": "^5.1.2", + "globals": "^13.6.0", + "ignore": "^4.0.6", + "import-fresh": "^3.0.0", + "imurmurhash": "^0.1.4", + "is-glob": "^4.0.0", + "js-yaml": "^3.13.1", + "json-stable-stringify-without-jsonify": "^1.0.1", + "levn": "^0.4.1", + "lodash.merge": "^4.6.2", + "minimatch": "^3.0.4", + "natural-compare": "^1.4.0", + "optionator": "^0.9.1", + "progress": "^2.0.0", + "regexpp": "^3.1.0", + "semver": "^7.2.1", + "strip-ansi": "^6.0.0", + "strip-json-comments": "^3.1.0", + "table": "^6.0.9", + "text-table": "^0.2.0", + "v8-compile-cache": "^2.0.3" + }, + "dependencies": { + "@babel/code-frame": { + "version": "7.12.11", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.12.11.tgz", + "integrity": "sha512-Zt1yodBx1UcyiePMSkWnU4hPqhwq7hGi2nFL1LeA3EUl+q2LQx16MISgJ0+z7dnmgvP9QtIleuETGOiOH1RcIw==", + "requires": { + "@babel/highlight": "^7.10.4" + } + }, + "cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "requires": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + } + }, + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + }, + "escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==" + }, + "eslint-utils": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/eslint-utils/-/eslint-utils-2.1.0.tgz", + "integrity": "sha512-w94dQYoauyvlDc43XnGB8lU3Zt713vNChgt4EWwhXAP2XkBvndfxF0AgIqKOOasjPIPzj9JqgwkwbCYD0/V3Zg==", + "requires": { + "eslint-visitor-keys": "^1.1.0" + }, + "dependencies": { + "eslint-visitor-keys": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz", + "integrity": "sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ==" + } + } + }, + "globals": { + "version": "13.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.14.0.tgz", + "integrity": "sha512-ERO68sOYwm5UuLvSJTY7w7NP2c8S4UcXs3X1GBX8cwOr+ShOcDBbCY5mH4zxz0jsYCdJ8ve8Mv9n2YGJMB1aeg==", + "requires": { + "type-fest": "^0.20.2" + } + }, + "ignore": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-4.0.6.tgz", + "integrity": "sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg==" + }, + "path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==" + }, + "shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "requires": { + "shebang-regex": "^3.0.0" + } + }, + "shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==" + }, + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "requires": { + "isexe": "^2.0.0" + } + } + } + }, + "eslint-config-react-app": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/eslint-config-react-app/-/eslint-config-react-app-6.0.0.tgz", + "integrity": "sha512-bpoAAC+YRfzq0dsTk+6v9aHm/uqnDwayNAXleMypGl6CpxI9oXXscVHo4fk3eJPIn+rsbtNetB4r/ZIidFIE8A==", + "requires": { + "confusing-browser-globals": "^1.0.10" + } + }, + "eslint-import-resolver-node": { + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.6.tgz", + "integrity": "sha512-0En0w03NRVMn9Uiyn8YRPDKvWjxCWkslUEhGNTdGx15RvPJYQ+lbOlqrlNI2vEAs4pDYK4f/HN2TbDmk5TP0iw==", + "requires": { + "debug": "^3.2.7", + "resolve": "^1.20.0" + } + }, + "eslint-module-utils": { + "version": "2.7.3", + "resolved": "https://registry.npmjs.org/eslint-module-utils/-/eslint-module-utils-2.7.3.tgz", + "integrity": "sha512-088JEC7O3lDZM9xGe0RerkOMd0EjFl+Yvd1jPWIkMT5u3H9+HC34mWWPnqPrN13gieT9pBOO+Qt07Nb/6TresQ==", + "requires": { + "debug": "^3.2.7", + "find-up": "^2.1.0" + }, + "dependencies": { + "find-up": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-2.1.0.tgz", + "integrity": "sha1-RdG35QbHF93UgndaK3eSCjwMV6c=", + "requires": { + "locate-path": "^2.0.0" + } + }, + "locate-path": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-2.0.0.tgz", + "integrity": "sha1-K1aLJl7slExtnA3pw9u7ygNUzY4=", + "requires": { + "p-locate": "^2.0.0", + "path-exists": "^3.0.0" + } + }, + "p-limit": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-1.3.0.tgz", + "integrity": "sha512-vvcXsLAJ9Dr5rQOPk7toZQZJApBl2K4J6dANSsEuh6QI41JYcsS/qhTGa9ErIUUgK3WNQoJYvylxvjqmiqEA9Q==", + "requires": { + "p-try": "^1.0.0" + } + }, + "p-locate": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-2.0.0.tgz", + "integrity": "sha1-IKAQOyIqcMj9OcwuWAaA893l7EM=", + "requires": { + "p-limit": "^1.1.0" + } + }, + "p-try": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-1.0.0.tgz", + "integrity": "sha1-y8ec26+P1CKOE/Yh8rGiN8GyB7M=" + }, + "path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha1-zg6+ql94yxiSXqfYENe1mwEP1RU=" + } + } + }, + "eslint-plugin-flowtype": { + "version": "5.10.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-flowtype/-/eslint-plugin-flowtype-5.10.0.tgz", + "integrity": "sha512-vcz32f+7TP+kvTUyMXZmCnNujBQZDNmcqPImw8b9PZ+16w1Qdm6ryRuYZYVaG9xRqqmAPr2Cs9FAX5gN+x/bjw==", + "requires": { + "lodash": "^4.17.15", + "string-natural-compare": "^3.0.1" + } + }, + "eslint-plugin-graphql": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-graphql/-/eslint-plugin-graphql-4.0.0.tgz", + "integrity": "sha512-d5tQm24YkVvCEk29ZR5ScsgXqAGCjKlMS8lx3mS7FS/EKsWbkvXQImpvic03EpMIvNTBW5e+2xnHzXB/VHNZJw==", + "requires": { + "@babel/runtime": "^7.10.0", + "graphql-config": "^3.0.2", + "lodash.flatten": "^4.4.0", + "lodash.without": "^4.4.0" + } + }, + "eslint-plugin-import": { + "version": "2.26.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-import/-/eslint-plugin-import-2.26.0.tgz", + "integrity": "sha512-hYfi3FXaM8WPLf4S1cikh/r4IxnO6zrhZbEGz2b660EJRbuxgpDS5gkCuYgGWg2xxh2rBuIr4Pvhve/7c31koA==", + "requires": { + "array-includes": "^3.1.4", + "array.prototype.flat": "^1.2.5", + "debug": "^2.6.9", + "doctrine": "^2.1.0", + "eslint-import-resolver-node": "^0.3.6", + "eslint-module-utils": "^2.7.3", + "has": "^1.0.3", + "is-core-module": "^2.8.1", + "is-glob": "^4.0.3", + "minimatch": "^3.1.2", + "object.values": "^1.1.5", + "resolve": "^1.22.0", + "tsconfig-paths": "^3.14.1" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "requires": { + "esutils": "^2.0.2" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "eslint-plugin-jsx-a11y": { + "version": "6.5.1", + "resolved": "https://registry.npmjs.org/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.5.1.tgz", + "integrity": "sha512-sVCFKX9fllURnXT2JwLN5Qgo24Ug5NF6dxhkmxsMEUZhXRcGg+X3e1JbJ84YePQKBl5E0ZjAH5Q4rkdcGY99+g==", + "requires": { + "@babel/runtime": "^7.16.3", + "aria-query": "^4.2.2", + "array-includes": "^3.1.4", + "ast-types-flow": "^0.0.7", + "axe-core": "^4.3.5", + "axobject-query": "^2.2.0", + "damerau-levenshtein": "^1.0.7", + "emoji-regex": "^9.2.2", + "has": "^1.0.3", + "jsx-ast-utils": "^3.2.1", + "language-tags": "^1.0.5", + "minimatch": "^3.0.4" + }, + "dependencies": { + "emoji-regex": { + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", + "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==" + } + } + }, + "eslint-plugin-react": { + "version": "7.29.4", + "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.29.4.tgz", + "integrity": "sha512-CVCXajliVh509PcZYRFyu/BoUEz452+jtQJq2b3Bae4v3xBUWPLCmtmBM+ZinG4MzwmxJgJ2M5rMqhqLVn7MtQ==", + "requires": { + "array-includes": "^3.1.4", + "array.prototype.flatmap": "^1.2.5", + "doctrine": "^2.1.0", + "estraverse": "^5.3.0", + "jsx-ast-utils": "^2.4.1 || ^3.0.0", + "minimatch": "^3.1.2", + "object.entries": "^1.1.5", + "object.fromentries": "^2.0.5", + "object.hasown": "^1.1.0", + "object.values": "^1.1.5", + "prop-types": "^15.8.1", + "resolve": "^2.0.0-next.3", + "semver": "^6.3.0", + "string.prototype.matchall": "^4.0.6" + }, + "dependencies": { + "doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "requires": { + "esutils": "^2.0.2" + } + }, + "estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==" + }, + "resolve": { + "version": "2.0.0-next.3", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-2.0.0-next.3.tgz", + "integrity": "sha512-W8LucSynKUIDu9ylraa7ueVZ7hc0uAgJBxVsQSKOXOyle8a93qXhcz+XAXZ8bIq2d6i4Ehddn6Evt+0/UwKk6Q==", + "requires": { + "is-core-module": "^2.2.0", + "path-parse": "^1.0.6" + } + }, + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "eslint-plugin-react-hooks": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.5.0.tgz", + "integrity": "sha512-8k1gRt7D7h03kd+SAAlzXkQwWK22BnK6GKZG+FJA6BAGy22CFvl8kCIXKpVux0cCxMWDQUPqSok0LKaZ0aOcCw==", + "requires": {} + }, + "eslint-scope": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-5.1.1.tgz", + "integrity": "sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw==", + "requires": { + "esrecurse": "^4.3.0", + "estraverse": "^4.1.1" + } + }, + "eslint-utils": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/eslint-utils/-/eslint-utils-3.0.0.tgz", + "integrity": "sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA==", + "requires": { + "eslint-visitor-keys": "^2.0.0" + } + }, + "eslint-visitor-keys": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-2.1.0.tgz", + "integrity": "sha512-0rSmRBzXgDzIsD6mGdJgevzgezI534Cer5L/vyMX0kHzT/jiB43jRhd9YUlMGYLQy2zprNmoT8qasCGtY+QaKw==" + }, + "eslint-webpack-plugin": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/eslint-webpack-plugin/-/eslint-webpack-plugin-2.6.0.tgz", + "integrity": "sha512-V+LPY/T3kur5QO3u+1s34VDTcRxjXWPUGM4hlmTb5DwVD0OQz631yGTxJZf4SpAqAjdbBVe978S8BJeHpAdOhQ==", + "requires": { + "@types/eslint": "^7.28.2", + "arrify": "^2.0.1", + "jest-worker": "^27.3.1", + "micromatch": "^4.0.4", + "normalize-path": "^3.0.0", + "schema-utils": "^3.1.1" + }, + "dependencies": { + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==" + }, + "jest-worker": { + "version": "27.5.1", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-27.5.1.tgz", + "integrity": "sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg==", + "requires": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^8.0.0" + } + }, + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + }, + "supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "requires": { + "has-flag": "^4.0.0" + } + } + } + }, + "espree": { + "version": "7.3.1", + "resolved": "https://registry.npmjs.org/espree/-/espree-7.3.1.tgz", + "integrity": "sha512-v3JCNCE64umkFpmkFGqzVKsOT0tN1Zr+ueqLZfpV1Ob8e+CEgPWa+OxCoGH3tnhimMKIaBm4m/vaRpJ/krRz2g==", + "requires": { + "acorn": "^7.4.0", + "acorn-jsx": "^5.3.1", + "eslint-visitor-keys": "^1.3.0" + }, + "dependencies": { + "eslint-visitor-keys": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz", + "integrity": "sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ==" + } + } + }, + "esprima": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/esprima/-/esprima-4.0.1.tgz", + "integrity": "sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A==" + }, + "esquery": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.4.0.tgz", + "integrity": "sha512-cCDispWt5vHHtwMY2YrAQ4ibFkAL8RbH5YGBnZBc90MolvvfkkQcJro/aZiAQUlQ3qgrYS6D6v8Gc5G5CQsc9w==", + "requires": { + "estraverse": "^5.1.0" + }, + "dependencies": { + "estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==" + } + } + }, + "esrecurse": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/esrecurse/-/esrecurse-4.3.0.tgz", + "integrity": "sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag==", + "requires": { + "estraverse": "^5.2.0" + }, + "dependencies": { + "estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==" + } + } + }, + "estraverse": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-4.3.0.tgz", + "integrity": "sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw==" + }, + "esutils": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/esutils/-/esutils-2.0.3.tgz", + "integrity": "sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g==" + }, + "etag": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/etag/-/etag-1.8.1.tgz", + "integrity": "sha1-Qa4u62XvpiJorr/qg6x9eSmbCIc=" + }, + "event-emitter": { + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/event-emitter/-/event-emitter-0.3.5.tgz", + "integrity": "sha1-34xp7vFkeSPHFXuc6DhAYQsCzDk=", + "requires": { + "d": "1", + "es5-ext": "~0.10.14" + } + }, + "event-source-polyfill": { + "version": "1.0.25", + "resolved": "https://registry.npmjs.org/event-source-polyfill/-/event-source-polyfill-1.0.25.tgz", + "integrity": "sha512-hQxu6sN1Eq4JjoI7ITdQeGGUN193A2ra83qC0Ltm9I2UJVAten3OFVN6k5RX4YWeCS0BoC8xg/5czOCIHVosQg==" + }, + "event-target-shim": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/event-target-shim/-/event-target-shim-5.0.1.tgz", + "integrity": "sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ==" + }, + "eventemitter3": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-3.1.2.tgz", + "integrity": "sha512-tvtQIeLVHjDkJYnzf2dgVMxfuSGJeM/7UCG17TT4EumTfNtF+0nebF/4zWOIkCreAbtNqhGEboB6BWrwqNaw4Q==" + }, + "events": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/events/-/events-3.3.0.tgz", + "integrity": "sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==" + }, + "execa": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", + "integrity": "sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg==", + "requires": { + "cross-spawn": "^7.0.3", + "get-stream": "^6.0.0", + "human-signals": "^2.1.0", + "is-stream": "^2.0.0", + "merge-stream": "^2.0.0", + "npm-run-path": "^4.0.1", + "onetime": "^5.1.2", + "signal-exit": "^3.0.3", + "strip-final-newline": "^2.0.0" + }, + "dependencies": { + "cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "requires": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + } + }, + "get-stream": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-6.0.1.tgz", + "integrity": "sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg==" + }, + "path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==" + }, + "shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "requires": { + "shebang-regex": "^3.0.0" + } + }, + "shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==" + }, + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "requires": { + "isexe": "^2.0.0" + } + } + } + }, + "exif-parser": { + "version": "0.1.12", + "resolved": "https://registry.npmjs.org/exif-parser/-/exif-parser-0.1.12.tgz", + "integrity": "sha1-WKnS1ywCwfbwKg70qRZicrd2CSI=" + }, + "expand-template": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/expand-template/-/expand-template-2.0.3.tgz", + "integrity": "sha512-XYfuKMvj4O35f/pOXLObndIRvyQ+/+6AhODh+OKWj9S9498pHHn/IMszH+gt0fBCRWMNfk1ZSp5x3AifmnI2vg==" + }, + "express": { + "version": "4.18.1", + "resolved": "https://registry.npmjs.org/express/-/express-4.18.1.tgz", + "integrity": "sha512-zZBcOX9TfehHQhtupq57OF8lFZ3UZi08Y97dwFCkD8p9d/d2Y3M+ykKcwaMDEL+4qyUolgBDX6AblpR3fL212Q==", + "requires": { + "accepts": "~1.3.8", + "array-flatten": "1.1.1", + "body-parser": "1.20.0", + "content-disposition": "0.5.4", + "content-type": "~1.0.4", + "cookie": "0.5.0", + "cookie-signature": "1.0.6", + "debug": "2.6.9", + "depd": "2.0.0", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "etag": "~1.8.1", + "finalhandler": "1.2.0", + "fresh": "0.5.2", + "http-errors": "2.0.0", + "merge-descriptors": "1.0.1", + "methods": "~1.1.2", + "on-finished": "2.4.1", + "parseurl": "~1.3.3", + "path-to-regexp": "0.1.7", + "proxy-addr": "~2.0.7", + "qs": "6.10.3", + "range-parser": "~1.2.1", + "safe-buffer": "5.2.1", + "send": "0.18.0", + "serve-static": "1.15.0", + "setprototypeof": "1.2.0", + "statuses": "2.0.1", + "type-is": "~1.6.18", + "utils-merge": "1.0.1", + "vary": "~1.1.2" + }, + "dependencies": { + "cookie": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.5.0.tgz", + "integrity": "sha512-YZ3GUyn/o8gfKJlnlX7g7xq4gyO6OSuhGPKaaGssGB2qgDUS0gPgtTvoyZLTt9Ab6dC4hfc9dV5arkvc/OCmrw==" + }, + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + } + } + }, + "express-graphql": { + "version": "0.12.0", + "resolved": "https://registry.npmjs.org/express-graphql/-/express-graphql-0.12.0.tgz", + "integrity": "sha512-DwYaJQy0amdy3pgNtiTDuGGM2BLdj+YO2SgbKoLliCfuHv3VVTt7vNG/ZqK2hRYjtYHE2t2KB705EU94mE64zg==", + "requires": { + "accepts": "^1.3.7", + "content-type": "^1.0.4", + "http-errors": "1.8.0", + "raw-body": "^2.4.1" + }, + "dependencies": { + "depd": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/depd/-/depd-1.1.2.tgz", + "integrity": "sha1-m81S4UwJd2PnSbJ0xDRu0uVgtak=" + }, + "http-errors": { + "version": "1.8.0", + "resolved": "https://registry.npmjs.org/http-errors/-/http-errors-1.8.0.tgz", + "integrity": "sha512-4I8r0C5JDhT5VkvI47QktDW75rNlGVsUf/8hzjCC/wkWI/jdTRmBb9aI7erSG82r1bjKY3F6k28WnsVxB1C73A==", + "requires": { + "depd": "~1.1.2", + "inherits": "2.0.4", + "setprototypeof": "1.2.0", + "statuses": ">= 1.5.0 < 2", + "toidentifier": "1.0.0" + } + }, + "statuses": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/statuses/-/statuses-1.5.0.tgz", + "integrity": "sha1-Fhx9rBd2Wf2YEfQ3cfqZOBR4Yow=" + }, + "toidentifier": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/toidentifier/-/toidentifier-1.0.0.tgz", + "integrity": "sha512-yaOH/Pk/VEhBWWTlhI+qXxDFXlejDGcQipMlyxda9nthulaxLZUNcUqFxokp0vcYnvteJln5FNQDRrxj3YcbVw==" + } + } + }, + "ext": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/ext/-/ext-1.6.0.tgz", + "integrity": "sha512-sdBImtzkq2HpkdRLtlLWDa6w4DX22ijZLKx8BMPUuKe1c5lbN6xwQDQCxSfxBQnHZ13ls/FH0MQZx/q/gr6FQg==", + "requires": { + "type": "^2.5.0" + }, + "dependencies": { + "type": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/type/-/type-2.6.0.tgz", + "integrity": "sha512-eiDBDOmkih5pMbo9OqsqPRGMljLodLcwd5XD5JbtNB0o89xZAwynY9EdCDsJU7LtcVCClu9DvM7/0Ep1hYX3EQ==" + } + } + }, + "extend": { + "version": "3.0.2", + "resolved": "https://registry.npmmirror.com/extend/-/extend-3.0.2.tgz", + "integrity": "sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==" + }, + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmmirror.com/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha512-zCnTtlxNoAiDc3gqY2aYAWFx7XWWiasuF2K8Me5WbN8otHKTUKBwjPtNpRs/rbUZm7KxWAaNj7P1a/p52GbVug==", + "requires": { + "is-extendable": "^0.1.0" + } + }, + "external-editor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/external-editor/-/external-editor-3.1.0.tgz", + "integrity": "sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew==", + "requires": { + "chardet": "^0.7.0", + "iconv-lite": "^0.4.24", + "tmp": "^0.0.33" + }, + "dependencies": { + "tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==", + "requires": { + "os-tmpdir": "~1.0.2" + } + } + } + }, + "extract-files": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/extract-files/-/extract-files-9.0.0.tgz", + "integrity": "sha512-CvdFfHkC95B4bBBk36hcEmvdR2awOdhhVUYH6S/zrVj3477zven/fJMYg7121h4T1xHZC+tetUpubpAhxwI7hQ==" + }, + "extsprintf": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/extsprintf/-/extsprintf-1.3.0.tgz", + "integrity": "sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==" + }, + "fast-deep-equal": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz", + "integrity": "sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q==" + }, + "fast-glob": { + "version": "3.2.11", + "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-3.2.11.tgz", + "integrity": "sha512-xrO3+1bxSo3ZVHAnqzyuewYT6aMFHRAd4Kcs92MAonjwQZLsK9d0SF1IyQ3k5PoirxTW0Oe/RqFgMQ6TcNE5Ew==", + "requires": { + "@nodelib/fs.stat": "^2.0.2", + "@nodelib/fs.walk": "^1.2.3", + "glob-parent": "^5.1.2", + "merge2": "^1.3.0", + "micromatch": "^4.0.4" + } + }, + "fast-json-stable-stringify": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz", + "integrity": "sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw==" + }, + "fast-levenshtein": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz", + "integrity": "sha1-PYpcZog6FqMMqGQ+hR8Zuqd5eRc=" + }, + "fastest-levenshtein": { + "version": "1.0.12", + "resolved": "https://registry.npmjs.org/fastest-levenshtein/-/fastest-levenshtein-1.0.12.tgz", + "integrity": "sha512-On2N+BpYJ15xIC974QNVuYGMOlEVt4s0EOI3wwMqOmK1fdDY+FN/zltPV8vosq4ad4c/gJ1KHScUn/6AWIgiow==" + }, + "fastq": { + "version": "1.13.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.13.0.tgz", + "integrity": "sha512-YpkpUnK8od0o1hmeSc7UUs/eB/vIPWJYjKck2QKIzAf71Vm1AAQ3EbuZB3g2JIy+pg+ERD0vqI79KyZiB2e2Nw==", + "requires": { + "reusify": "^1.0.4" + } + }, + "fb-watchman": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/fb-watchman/-/fb-watchman-2.0.1.tgz", + "integrity": "sha512-DkPJKQeY6kKwmuMretBhr7G6Vodr7bFwDYTXIkfG1gjvNpaxBTQV3PbXg6bR1c1UP4jPOX0jHUbbHANL9vRjVg==", + "requires": { + "bser": "2.1.1" + } + }, + "fbjs": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/fbjs/-/fbjs-3.0.4.tgz", + "integrity": "sha512-ucV0tDODnGV3JCnnkmoszb5lf4bNpzjv80K41wd4k798Etq+UYD0y0TIfalLjZoKgjive6/adkRnszwapiDgBQ==", + "requires": { + "cross-fetch": "^3.1.5", + "fbjs-css-vars": "^1.0.0", + "loose-envify": "^1.0.0", + "object-assign": "^4.1.0", + "promise": "^7.1.1", + "setimmediate": "^1.0.5", + "ua-parser-js": "^0.7.30" + } + }, + "fbjs-css-vars": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/fbjs-css-vars/-/fbjs-css-vars-1.0.2.tgz", + "integrity": "sha512-b2XGFAFdWZWg0phtAWLHCk836A1Xann+I+Dgd3Gk64MHKZO44FfoD1KxyvbSh0qZsIoXQGGlVztIY+oitJPpRQ==" + }, + "fd": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/fd/-/fd-0.0.3.tgz", + "integrity": "sha512-iAHrIslQb3U68OcMSP0kkNWabp7sSN6d2TBSb2JO3gcLJVDd4owr/hKM4SFJovFOUeeXeItjYgouEDTMWiVAnA==" + }, + "figures": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/figures/-/figures-3.2.0.tgz", + "integrity": "sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg==", + "requires": { + "escape-string-regexp": "^1.0.5" + } + }, + "file-entry-cache": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-6.0.1.tgz", + "integrity": "sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg==", + "requires": { + "flat-cache": "^3.0.4" + } + }, + "file-loader": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/file-loader/-/file-loader-6.2.0.tgz", + "integrity": "sha512-qo3glqyTa61Ytg4u73GultjHGjdRyig3tG6lPtyX/jOEJvHif9uB0/OCI2Kif6ctF3caQTW2G5gym21oAsI4pw==", + "requires": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "file-type": { + "version": "16.5.3", + "resolved": "https://registry.npmjs.org/file-type/-/file-type-16.5.3.tgz", + "integrity": "sha512-uVsl7iFhHSOY4bEONLlTK47iAHtNsFHWP5YE4xJfZ4rnX7S1Q3wce09XgqSC7E/xh8Ncv/be1lNoyprlUH/x6A==", + "requires": { + "readable-web-to-node-stream": "^3.0.0", + "strtok3": "^6.2.4", + "token-types": "^4.1.1" + } + }, + "filename-reserved-regex": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/filename-reserved-regex/-/filename-reserved-regex-2.0.0.tgz", + "integrity": "sha1-q/c9+rc10EVECr/qLZHzieu/oik=" + }, + "filenamify": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/filenamify/-/filenamify-4.3.0.tgz", + "integrity": "sha512-hcFKyUG57yWGAzu1CMt/dPzYZuv+jAJUT85bL8mrXvNe6hWj6yEHEc4EdcgiA6Z3oi1/9wXJdZPXF2dZNgwgOg==", + "requires": { + "filename-reserved-regex": "^2.0.0", + "strip-outer": "^1.0.1", + "trim-repeated": "^1.0.0" + } + }, + "filesize": { + "version": "8.0.7", + "resolved": "https://registry.npmjs.org/filesize/-/filesize-8.0.7.tgz", + "integrity": "sha512-pjmC+bkIF8XI7fWaH8KxHcZL3DPybs1roSKP4rKDvy20tAWwIObE4+JIseG2byfGKhud5ZnM4YSGKBz7Sh0ndQ==" + }, + "fill-range": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", + "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "requires": { + "to-regex-range": "^5.0.1" + } + }, + "filter-obj": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/filter-obj/-/filter-obj-1.1.0.tgz", + "integrity": "sha1-mzERErxsYSehbgFsbF1/GeCAXFs=" + }, + "finalhandler": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.2.0.tgz", + "integrity": "sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg==", + "requires": { + "debug": "2.6.9", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "on-finished": "2.4.1", + "parseurl": "~1.3.3", + "statuses": "2.0.1", + "unpipe": "~1.0.0" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "find-cache-dir": { + "version": "3.3.2", + "resolved": "https://registry.npmjs.org/find-cache-dir/-/find-cache-dir-3.3.2.tgz", + "integrity": "sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig==", + "requires": { + "commondir": "^1.0.1", + "make-dir": "^3.0.2", + "pkg-dir": "^4.1.0" + } + }, + "find-up": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", + "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "requires": { + "locate-path": "^5.0.0", + "path-exists": "^4.0.0" + } + }, + "flat-cache": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-3.0.4.tgz", + "integrity": "sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg==", + "requires": { + "flatted": "^3.1.0", + "rimraf": "^3.0.2" + } + }, + "flatted": { + "version": "3.2.5", + "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.2.5.tgz", + "integrity": "sha512-WIWGi2L3DyTUvUrwRKgGi9TwxQMUEqPOPQBVi71R96jZXJdFskXEmf54BoZaS1kknGODoIGASGEzBUYdyMCBJg==" + }, + "follow-redirects": { + "version": "1.15.0", + "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.0.tgz", + "integrity": "sha512-aExlJShTV4qOUOL7yF1U5tvLCB0xQuudbf6toyYA0E/acBNw71mvjFTnLaRp50aQaYocMR0a/RMMBIHeZnGyjQ==" + }, + "forever-agent": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/forever-agent/-/forever-agent-0.6.1.tgz", + "integrity": "sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==" + }, + "fork-ts-checker-webpack-plugin": { + "version": "6.5.2", + "resolved": "https://registry.npmjs.org/fork-ts-checker-webpack-plugin/-/fork-ts-checker-webpack-plugin-6.5.2.tgz", + "integrity": "sha512-m5cUmF30xkZ7h4tWUgTAcEaKmUW7tfyUyTqNNOz7OxWJ0v1VWKTcOvH8FWHUwSjlW/356Ijc9vi3XfcPstpQKA==", + "requires": { + "@babel/code-frame": "^7.8.3", + "@types/json-schema": "^7.0.5", + "chalk": "^4.1.0", + "chokidar": "^3.4.2", + "cosmiconfig": "^6.0.0", + "deepmerge": "^4.2.2", + "fs-extra": "^9.0.0", + "glob": "^7.1.6", + "memfs": "^3.1.2", + "minimatch": "^3.0.4", + "schema-utils": "2.7.0", + "semver": "^7.3.2", + "tapable": "^1.0.0" + }, + "dependencies": { + "cosmiconfig": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-6.0.0.tgz", + "integrity": "sha512-xb3ZL6+L8b9JLLCx3ZdoZy4+2ECphCMo2PwqgP1tlfVq6M6YReyzBJtvWWtbDSpNr9hn96pkCiZqUcFEc+54Qg==", + "requires": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.1.0", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.7.2" + } + }, + "fs-extra": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-9.1.0.tgz", + "integrity": "sha512-hcg3ZmepS30/7BSFqRvoo3DOMQu7IjqxO5nCDt+zM9XWjb33Wg7ziNT+Qvqbuc3+gWpzO02JubVyk2G4Zvo1OQ==", + "requires": { + "at-least-node": "^1.0.0", + "graceful-fs": "^4.2.0", + "jsonfile": "^6.0.1", + "universalify": "^2.0.0" + } + }, + "schema-utils": { + "version": "2.7.0", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-2.7.0.tgz", + "integrity": "sha512-0ilKFI6QQF5nxDZLFn2dMjvc4hjg/Wkg7rHd3jK6/A4a1Hl9VFdQWvgB1UMGoU94pad1P/8N7fMcEnLnSiju8A==", + "requires": { + "@types/json-schema": "^7.0.4", + "ajv": "^6.12.2", + "ajv-keywords": "^3.4.1" + } + }, + "tapable": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-1.1.3.tgz", + "integrity": "sha512-4WK/bYZmj8xLr+HUCODHGF1ZFzsYffasLUgEiMBY4fgtltdO6B4WJtlSbPaDTLpYTcGVwM2qLnFTICEcNxs3kA==" + } + } + }, + "form-data": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", + "integrity": "sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==", + "requires": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + } + }, + "forwarded": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/forwarded/-/forwarded-0.2.0.tgz", + "integrity": "sha512-buRG0fpBtRHSTCOASe6hD258tEubFoRLb4ZNA6NxMVHNw2gOcwHo9wyablzMzOA5z9xA9L1KNjk/Nt6MT9aYow==" + }, + "fraction.js": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/fraction.js/-/fraction.js-4.2.0.tgz", + "integrity": "sha512-MhLuK+2gUcnZe8ZHlaaINnQLl0xRIGRfcGk2yl8xoQAfHrSsL3rYu6FCmBdkdbhc9EPlwyGHewaRsvwRMJtAlA==" + }, + "fresh": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/fresh/-/fresh-0.5.2.tgz", + "integrity": "sha1-PYyt2Q2XZWn6g1qx+OSyOhBWBac=" + }, + "fs-constants": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs-constants/-/fs-constants-1.0.0.tgz", + "integrity": "sha512-y6OAwoSIf7FyjMIv94u+b5rdheZEjzR63GTyZJm5qh4Bi+2YgwLCcI/fPFZkL5PSixOt6ZNKm+w+Hfp/Bciwow==" + }, + "fs-exists-cached": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs-exists-cached/-/fs-exists-cached-1.0.0.tgz", + "integrity": "sha1-zyVVTKBQ3EmuZla0HeQiWJidy84=" + }, + "fs-extra": { + "version": "10.1.0", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-10.1.0.tgz", + "integrity": "sha512-oRXApq54ETRj4eMiFzGnHWGy+zo5raudjuxN0b8H7s/RU2oW0Wvsx9O0ACRN/kRq9E8Vu/ReskGB5o3ji+FzHQ==", + "requires": { + "graceful-fs": "^4.2.0", + "jsonfile": "^6.0.1", + "universalify": "^2.0.0" + } + }, + "fs-minipass": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/fs-minipass/-/fs-minipass-2.1.0.tgz", + "integrity": "sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg==", + "requires": { + "minipass": "^3.0.0" + } + }, + "fs-monkey": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/fs-monkey/-/fs-monkey-1.0.3.tgz", + "integrity": "sha512-cybjIfiiE+pTWicSCLFHSrXZ6EilF30oh91FDP9S2B051prEa7QWfrVTQm10/dDpswBDXZugPa1Ogu8Yh+HV0Q==" + }, + "fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=" + }, + "fsevents": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.2.tgz", + "integrity": "sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA==", + "optional": true + }, + "function-bind": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz", + "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==" + }, + "function.prototype.name": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/function.prototype.name/-/function.prototype.name-1.1.5.tgz", + "integrity": "sha512-uN7m/BzVKQnCUF/iW8jYea67v++2u7m5UgENbHRtdDVclOUP+FMPlCNdmk0h/ysGyo2tavMJEDqJAkJdRa1vMA==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.0", + "functions-have-names": "^1.2.2" + } + }, + "functional-red-black-tree": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz", + "integrity": "sha1-GwqzvVU7Kg1jmdKcDj6gslIHgyc=" + }, + "functions-have-names": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/functions-have-names/-/functions-have-names-1.2.3.tgz", + "integrity": "sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==" + }, + "gatsby": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby/-/gatsby-4.14.0.tgz", + "integrity": "sha512-7Nwy+1QHligFcV94I/Gvs6rBHu+GN5L2s2FoSU2h0vs+p3ys0UzpBhaqxqiPZynfCWX3vRUquDvoeSR1ALYgOg==", + "requires": { + "@babel/code-frame": "^7.14.0", + "@babel/core": "^7.15.5", + "@babel/eslint-parser": "^7.15.4", + "@babel/helper-plugin-utils": "^7.14.5", + "@babel/parser": "^7.15.5", + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "@babel/types": "^7.15.4", + "@gatsbyjs/reach-router": "^1.3.6", + "@gatsbyjs/webpack-hot-middleware": "^2.25.2", + "@graphql-codegen/add": "^3.1.1", + "@graphql-codegen/core": "^2.5.1", + "@graphql-codegen/plugin-helpers": "^2.4.2", + "@graphql-codegen/typescript": "^2.4.8", + "@graphql-codegen/typescript-operations": "^2.3.5", + "@graphql-tools/code-file-loader": "^7.2.14", + "@graphql-tools/load": "^7.5.10", + "@nodelib/fs.walk": "^1.2.8", + "@parcel/core": "^2.3.2", + "@pmmmwh/react-refresh-webpack-plugin": "^0.4.3", + "@types/http-proxy": "^1.17.7", + "@typescript-eslint/eslint-plugin": "^4.33.0", + "@typescript-eslint/parser": "^4.33.0", + "@vercel/webpack-asset-relocator-loader": "^1.7.0", + "address": "1.1.2", + "anser": "^2.1.0", + "autoprefixer": "^10.4.0", + "axios": "^0.21.1", + "babel-loader": "^8.2.3", + "babel-plugin-add-module-exports": "^1.0.4", + "babel-plugin-dynamic-import-node": "^2.3.3", + "babel-plugin-lodash": "^3.3.4", + "babel-plugin-remove-graphql-queries": "^4.14.0", + "babel-preset-gatsby": "^2.14.0", + "better-opn": "^2.1.1", + "bluebird": "^3.7.2", + "body-parser": "^1.19.0", + "browserslist": "^4.17.5", + "cache-manager": "^2.11.1", + "chalk": "^4.1.2", + "chokidar": "^3.5.2", + "common-tags": "^1.8.0", + "compression": "^1.7.4", + "cookie": "^0.4.1", + "core-js": "^3.22.3", + "cors": "^2.8.5", + "css-loader": "^5.2.7", + "css-minimizer-webpack-plugin": "^2.0.0", + "css.escape": "^1.5.1", + "date-fns": "^2.25.0", + "debug": "^3.2.7", + "deepmerge": "^4.2.2", + "detect-port": "^1.3.0", + "devcert": "^1.2.0", + "dotenv": "^8.6.0", + "enhanced-resolve": "^5.8.3", + "eslint": "^7.32.0", + "eslint-config-react-app": "^6.0.0", + "eslint-plugin-flowtype": "^5.10.0", + "eslint-plugin-graphql": "^4.0.0", + "eslint-plugin-import": "^2.26.0", + "eslint-plugin-jsx-a11y": "^6.5.1", + "eslint-plugin-react": "^7.29.4", + "eslint-plugin-react-hooks": "^4.5.0", + "eslint-webpack-plugin": "^2.6.0", + "event-source-polyfill": "1.0.25", + "execa": "^5.1.1", + "express": "^4.17.1", + "express-graphql": "^0.12.0", + "fastest-levenshtein": "^1.0.12", + "fastq": "^1.13.0", + "file-loader": "^6.2.0", + "find-cache-dir": "^3.3.2", + "fs-exists-cached": "1.0.0", + "fs-extra": "^10.1.0", + "gatsby-cli": "^4.14.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-graphiql-explorer": "^2.14.0", + "gatsby-legacy-polyfills": "^2.14.0", + "gatsby-link": "^4.14.0", + "gatsby-page-utils": "^2.14.0", + "gatsby-parcel-config": "^0.5.0", + "gatsby-plugin-page-creator": "^4.14.0", + "gatsby-plugin-typescript": "^4.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-react-router-scroll": "^5.14.0", + "gatsby-sharp": "^0.8.0", + "gatsby-telemetry": "^3.14.0", + "gatsby-worker": "^1.14.0", + "glob": "^7.2.0", + "globby": "^11.1.0", + "got": "^11.8.2", + "graphql": "^15.7.2", + "graphql-compose": "^9.0.7", + "graphql-playground-middleware-express": "^1.7.22", + "hasha": "^5.2.2", + "http-proxy": "^1.18.1", + "invariant": "^2.2.4", + "is-relative": "^1.0.0", + "is-relative-url": "^3.0.0", + "joi": "^17.4.2", + "json-loader": "^0.5.7", + "latest-version": "5.1.0", + "lmdb": "~2.2.3", + "lodash": "^4.17.21", + "md5-file": "^5.0.0", + "meant": "^1.0.3", + "memoizee": "^0.4.15", + "micromatch": "^4.0.4", + "mime": "^2.5.2", + "mini-css-extract-plugin": "1.6.2", + "mitt": "^1.2.0", + "moment": "^2.29.1", + "multer": "^1.4.3", + "node-fetch": "^2.6.6", + "normalize-path": "^3.0.0", + "null-loader": "^4.0.1", + "opentracing": "^0.14.5", + "p-defer": "^3.0.0", + "parseurl": "^1.3.3", + "physical-cpu-count": "^2.0.0", + "platform": "^1.3.6", + "postcss": "^8.3.11", + "postcss-flexbugs-fixes": "^5.0.2", + "postcss-loader": "^5.3.0", + "prompts": "^2.4.2", + "prop-types": "^15.7.2", + "query-string": "^6.14.1", + "raw-loader": "^4.0.2", + "react-dev-utils": "^12.0.1", + "react-refresh": "^0.9.0", + "redux": "4.1.2", + "redux-thunk": "^2.4.0", + "resolve-from": "^5.0.0", + "semver": "^7.3.7", + "shallow-compare": "^1.2.2", + "signal-exit": "^3.0.5", + "slugify": "^1.6.1", + "socket.io": "3.1.2", + "socket.io-client": "3.1.3", + "source-map": "^0.7.3", + "source-map-support": "^0.5.20", + "st": "^2.0.0", + "stack-trace": "^0.0.10", + "string-similarity": "^1.2.2", + "strip-ansi": "^6.0.1", + "style-loader": "^2.0.0", + "terser-webpack-plugin": "^5.2.4", + "tmp": "^0.2.1", + "true-case-path": "^2.2.1", + "type-of": "^2.0.1", + "url-loader": "^4.1.1", + "uuid": "^8.3.2", + "webpack": "^5.61.0", + "webpack-dev-middleware": "^4.3.0", + "webpack-merge": "^5.8.0", + "webpack-stats-plugin": "^1.0.3", + "webpack-virtual-modules": "^0.3.2", + "xstate": "^4.26.0", + "yaml-loader": "^0.6.0" + }, + "dependencies": { + "gatsby-cli": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-cli/-/gatsby-cli-4.14.0.tgz", + "integrity": "sha512-jmLhrBNguZM8ldKpt1dmxbEZ4j/OtEdE1IpUCHoLGoCIZ7QGtleA2WHhn0R4GnoY0FVP7+pGWcmPpBXo63DBXA==", + "requires": { + "@babel/code-frame": "^7.14.0", + "@babel/core": "^7.15.5", + "@babel/generator": "^7.16.8", + "@babel/helper-plugin-utils": "^7.16.7", + "@babel/preset-typescript": "^7.16.7", + "@babel/runtime": "^7.15.4", + "@babel/template": "^7.16.7", + "@babel/types": "^7.16.8", + "@types/common-tags": "^1.8.1", + "better-opn": "^2.1.1", + "boxen": "^5.1.2", + "chalk": "^4.1.2", + "clipboardy": "^2.3.0", + "common-tags": "^1.8.2", + "configstore": "^5.0.1", + "convert-hrtime": "^3.0.0", + "create-gatsby": "^2.14.0", + "envinfo": "^7.8.1", + "execa": "^5.1.1", + "fs-exists-cached": "^1.0.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-telemetry": "^3.14.0", + "hosted-git-info": "^3.0.8", + "is-valid-path": "^0.1.1", + "joi": "^17.4.2", + "lodash": "^4.17.21", + "meant": "^1.0.3", + "node-fetch": "^2.6.6", + "opentracing": "^0.14.5", + "pretty-error": "^2.1.2", + "progress": "^2.0.3", + "prompts": "^2.4.2", + "redux": "4.1.2", + "resolve-cwd": "^3.0.0", + "semver": "^7.3.7", + "signal-exit": "^3.0.6", + "source-map": "0.7.3", + "stack-trace": "^0.0.10", + "strip-ansi": "^6.0.1", + "update-notifier": "^5.1.0", + "uuid": "3.4.0", + "yargs": "^15.4.1", + "yoga-layout-prebuilt": "^1.10.0", + "yurnalist": "^2.1.0" + }, + "dependencies": { + "uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==" + } + } + } + } + }, + "gatsby-core-utils": { + "version": "3.14.0", + "resolved": "https://registry.npmjs.org/gatsby-core-utils/-/gatsby-core-utils-3.14.0.tgz", + "integrity": "sha512-JavHwcX5L+ZRoL5FKhYex3JfbwwS0273YTpf8y8SRKsObD8H+bbLOUlbOjASpqy+IU3dW+r76gT1dQdaqeH9Og==", + "requires": { + "@babel/runtime": "^7.15.4", + "ci-info": "2.0.0", + "configstore": "^5.0.1", + "fastq": "^1.13.0", + "file-type": "^16.5.3", + "fs-extra": "^10.1.0", + "got": "^11.8.3", + "import-from": "^4.0.0", + "lmdb": "^2.2.6", + "lock": "^1.1.0", + "node-object-hash": "^2.3.10", + "proper-lockfile": "^4.1.2", + "resolve-from": "^5.0.0", + "tmp": "^0.2.1", + "xdg-basedir": "^4.0.0" + } + }, + "gatsby-graphiql-explorer": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-graphiql-explorer/-/gatsby-graphiql-explorer-2.14.0.tgz", + "integrity": "sha512-J71G+WtSRmykmmdqYYGz5CYC6zToTmJqyywKpN83aZF2z7h7Ab2FHBuiP84KIlF2xpSxsk26puZ40TIHOGP2yw==", + "requires": { + "@babel/runtime": "^7.15.4" + } + }, + "gatsby-legacy-polyfills": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-legacy-polyfills/-/gatsby-legacy-polyfills-2.14.0.tgz", + "integrity": "sha512-OcJrY9eqiHtU8bi1zOiaO+wXZv+W/HOR0oP+5IvmWBIiLl4M+ln/z6PJcqk2fnfIK51zyzARvhPXAakDs5JE4w==", + "requires": { + "@babel/runtime": "^7.15.4", + "core-js-compat": "3.9.0" + }, + "dependencies": { + "core-js-compat": { + "version": "3.9.0", + "resolved": "https://registry.npmjs.org/core-js-compat/-/core-js-compat-3.9.0.tgz", + "integrity": "sha512-YK6fwFjCOKWwGnjFUR3c544YsnA/7DoLL0ysncuOJ4pwbriAtOpvM2bygdlcXbvQCQZ7bBU9CL4t7tGl7ETRpQ==", + "requires": { + "browserslist": "^4.16.3", + "semver": "7.0.0" + } + }, + "semver": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.0.0.tgz", + "integrity": "sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A==" + } + } + }, + "gatsby-link": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-link/-/gatsby-link-4.14.0.tgz", + "integrity": "sha512-zaMhjalc5RL4knxcJJyBee3wfIjZzcQGTdytHjS1HwOLV/cewIQZFCKIeScgg/623bRFCqQAqB3dzcwAbxhq3A==", + "requires": { + "@babel/runtime": "^7.15.4", + "@types/reach__router": "^1.3.10", + "gatsby-page-utils": "^2.14.0", + "prop-types": "^15.8.1" + } + }, + "gatsby-page-utils": { + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/gatsby-page-utils/-/gatsby-page-utils-2.14.0.tgz", + "integrity": "sha512-7o2NEJIMFPmcwwynED9osHsL4byIRNolYUUijxzNgnVQfvRnmuSZZABoC3nF0uInVvAQcE496vGGj284SfyOLw==", + "requires": { + "@babel/runtime": "^7.15.4", + "bluebird": "^3.7.2", + "chokidar": "^3.5.2", + "fs-exists-cached": "^1.0.0", + "gatsby-core-utils": "^3.14.0", + "glob": "^7.2.0", + "lodash": "^4.17.21", + "micromatch": "^4.0.5" + } + }, + "gatsby-parcel-config": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/gatsby-parcel-config/-/gatsby-parcel-config-0.5.0.tgz", + "integrity": "sha512-Ff4MD1y9+tYLlzQ377TiW79L0+PQxTc8FKm+l6bYDs9LDmPf4I+tshIIJnQEJE7aLuR66Ow9qSdluZj2Df2msA==", + "requires": { + "@gatsbyjs/parcel-namer-relative-to-cwd": "0.0.2", + "@parcel/bundler-default": "^2.3.2", + "@parcel/compressor-raw": "^2.3.2", + "@parcel/namer-default": "^2.3.2", + "@parcel/optimizer-terser": "^2.3.2", + "@parcel/packager-js": "^2.3.2", + "@parcel/packager-raw": "^2.3.2", + "@parcel/reporter-dev-server": "^2.3.2", + "@parcel/resolver-default": "^2.3.2", + "@parcel/runtime-browser-hmr": "^2.3.2", + "@parcel/runtime-js": "^2.3.2", + "@parcel/runtime-react-refresh": "^2.3.2", + "@parcel/runtime-service-worker": "^2.3.2", + "@parcel/transformer-js": "^2.3.2", + "@parcel/transformer-json": "^2.3.2", + "@parcel/transformer-raw": "^2.3.2", + "@parcel/transformer-react-refresh-wrap": "^2.3.2" + } + }, + "gatsby-plugin-gatsby-cloud": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-gatsby-cloud/-/gatsby-plugin-gatsby-cloud-4.14.0.tgz", + "integrity": "sha512-9xG+9L5I/Dj2PgpXeTdwFu4FhEjbpAHJaQJuxi9Wz2h4O3ix02nkXmvcDrX3JC/peOQ4zFfHh/mTy2jHl2pzow==", + "requires": { + "@babel/runtime": "^7.15.4", + "date-fns": "^2.28.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-telemetry": "^3.14.0", + "js-cookie": "^3.0.1", + "kebab-hash": "^0.1.2", + "lodash": "^4.17.21", + "webpack-assets-manifest": "^5.1.0" + } + }, + "gatsby-plugin-image": { + "version": "2.14.1", + "resolved": "https://registry.npmjs.org/gatsby-plugin-image/-/gatsby-plugin-image-2.14.1.tgz", + "integrity": "sha512-9ez7/MqjZLwvFqY37miFO3IAcqF3rkhUMyrwct+JMk2c1AcXgGyG1acgKGkJU6maiQoIqSKtyzjoQD8+o3f0Hg==", + "requires": { + "@babel/code-frame": "^7.14.0", + "@babel/parser": "^7.15.5", + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "babel-jsx-utils": "^1.1.0", + "babel-plugin-remove-graphql-queries": "^4.14.0", + "camelcase": "^5.3.1", + "chokidar": "^3.5.2", + "common-tags": "^1.8.2", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "objectFitPolyfill": "^2.3.5", + "prop-types": "^15.8.1" + } + }, + "gatsby-plugin-manifest": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-manifest/-/gatsby-plugin-manifest-4.14.0.tgz", + "integrity": "sha512-vwN5ZasR6/I9Xd1Ar3+UhMaYN6EU7PJhJKY6aQMAtG1Qxva5lDjmrWNzUlm8NHL/XmB4VSS+A4TZUZHyoygZ7Q==", + "requires": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0", + "gatsby-plugin-utils": "^3.8.0", + "semver": "^7.3.7", + "sharp": "^0.30.3" + } + }, + "gatsby-plugin-mantine": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/gatsby-plugin-mantine/-/gatsby-plugin-mantine-4.0.0.tgz", + "integrity": "sha512-7Cc0f01H7DEsuLhe+0Qst0oKlZB/jhxRCL4WMW7m+fSc3s57fBrL44ObhFlWCWQtLJkSsOx9Fl4Se4FN+gwySA==", + "requires": { + "@mantine/ssr": ">=3.6.14" + } + }, + "gatsby-plugin-offline": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-offline/-/gatsby-plugin-offline-5.14.0.tgz", + "integrity": "sha512-7ClwsPdX3ZEaR6BAKvXVRn/lh5X08i7xugXpI+qDXWfCFOmuWh3iy4fl1ESPxWRhem99bvv5ocqhGQ0Q1tAbHQ==", + "requires": { + "@babel/runtime": "^7.15.4", + "cheerio": "^1.0.0-rc.10", + "gatsby-core-utils": "^3.14.0", + "glob": "^7.2.0", + "idb-keyval": "^3.2.0", + "lodash": "^4.17.21", + "workbox-build": "^4.3.1" + } + }, + "gatsby-plugin-page-creator": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-page-creator/-/gatsby-plugin-page-creator-4.14.0.tgz", + "integrity": "sha512-bDKnnXJ0H7fUNgEqvUWhPW6N3HkYmQbUQunCYTNl0tON9rLm8Rb0PwdlpCTeg20DOORMhIJZ3Uk6o2KMTGVlAQ==", + "requires": { + "@babel/runtime": "^7.15.4", + "@babel/traverse": "^7.15.4", + "@sindresorhus/slugify": "^1.1.2", + "chokidar": "^3.5.2", + "fs-exists-cached": "^1.0.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-page-utils": "^2.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-telemetry": "^3.14.0", + "globby": "^11.1.0", + "lodash": "^4.17.21" + } + }, + "gatsby-plugin-react-helmet": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-react-helmet/-/gatsby-plugin-react-helmet-5.14.0.tgz", + "integrity": "sha512-Loy0fUtVDhEuvv1nCpsyyUpAosGRUB3g5GzHvV3oqxcGgrpTgeXTPH64drXuRKcHyJbH1vgUOuPdlIEzXDbQMQ==", + "requires": { + "@babel/runtime": "^7.15.4" + } + }, + "gatsby-plugin-sass": { + "version": "5.14.0", + "resolved": "https://registry.npmmirror.com/gatsby-plugin-sass/-/gatsby-plugin-sass-5.14.0.tgz", + "integrity": "sha512-gRts8Tfb1AyNey+xPTRVFlc+agLM7DDZlh4V665RsWja9vF/bQlWwvX1DjmHgYkeVobh0dMyd9QV4TyDv9Uurw==", + "requires": { + "@babel/runtime": "^7.15.4", + "resolve-url-loader": "^3.1.4", + "sass-loader": "^10.1.1" + } + }, + "gatsby-plugin-sharp": { + "version": "4.14.1", + "resolved": "https://registry.npmjs.org/gatsby-plugin-sharp/-/gatsby-plugin-sharp-4.14.1.tgz", + "integrity": "sha512-izYl9XcnWdAY+rDtZtnKfzZ/djBioGSDg/k84cgLVbTzLB29jgqipw3PkjV7IlvmJmlw2v78YCHO5mk3bfcitg==", + "requires": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "async": "^3.2.3", + "bluebird": "^3.7.2", + "debug": "^4.3.4", + "filenamify": "^4.3.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-plugin-utils": "^3.8.0", + "gatsby-telemetry": "^3.14.0", + "got": "^11.8.3", + "lodash": "^4.17.21", + "mini-svg-data-uri": "^1.4.4", + "probe-image-size": "^7.2.3", + "progress": "^2.0.3", + "semver": "^7.3.7", + "sharp": "^0.30.3", + "svgo": "1.3.2", + "uuid": "3.4.0" + }, + "dependencies": { + "async": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/async/-/async-3.2.3.tgz", + "integrity": "sha512-spZRyzKL5l5BZQrr/6m/SqFdBN0q3OCI0f9rjfBzCMBIP4p75P620rR3gTmaksNOhmzgdxcaxdNfMy6anrbM0g==" + }, + "chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "css-select": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/css-select/-/css-select-2.1.0.tgz", + "integrity": "sha512-Dqk7LQKpwLoH3VovzZnkzegqNSuAziQyNZUcrdDM401iY+R5NkGBXGmtO05/yaXQziALuPogeG0b7UAgjnTJTQ==", + "requires": { + "boolbase": "^1.0.0", + "css-what": "^3.2.1", + "domutils": "^1.7.0", + "nth-check": "^1.0.2" + } + }, + "css-tree": { + "version": "1.0.0-alpha.37", + "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.0.0-alpha.37.tgz", + "integrity": "sha512-DMxWJg0rnz7UgxKT0Q1HU/L9BeJI0M6ksor0OgqOnF+aRCDWg/N2641HmVyU9KVIu0OVVWOb2IpC9A+BJRnejg==", + "requires": { + "mdn-data": "2.0.4", + "source-map": "^0.6.1" + } + }, + "css-what": { + "version": "3.4.2", + "resolved": "https://registry.npmjs.org/css-what/-/css-what-3.4.2.tgz", + "integrity": "sha512-ACUm3L0/jiZTqfzRM3Hi9Q8eZqd6IK37mMWPLz9PJxkLWllYeRf+EHUSHYEtFop2Eqytaq1FizFVh7XfBnXCDQ==" + }, + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + }, + "dom-serializer": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.2.2.tgz", + "integrity": "sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g==", + "requires": { + "domelementtype": "^2.0.1", + "entities": "^2.0.0" + }, + "dependencies": { + "domelementtype": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-2.3.0.tgz", + "integrity": "sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw==" + } + } + }, + "domelementtype": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", + "integrity": "sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w==" + }, + "domutils": { + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.7.0.tgz", + "integrity": "sha512-Lgd2XcJ/NjEw+7tFvfKxOzCYKZsdct5lczQ2ZaQY8Djz7pfAD3Gbp8ySJWtreII/vDlMVmxwa6pHmdxIYgttDg==", + "requires": { + "dom-serializer": "0", + "domelementtype": "1" + } + }, + "mdn-data": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-2.0.4.tgz", + "integrity": "sha512-iV3XNKw06j5Q7mi6h+9vbx23Tv7JkjEVgKHW4pimwyDGWm0OIQntJJ+u1C6mg6mK1EaTv42XQ7w76yuzH7M2cA==" + }, + "nth-check": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/nth-check/-/nth-check-1.0.2.tgz", + "integrity": "sha512-WeBOdju8SnzPN5vTUJYxYUxLeXpCaVP5i5e0LF8fg7WORF2Wd7wFX/pk0tYZk7s8T+J7VLy0Da6J1+wCT0AtHg==", + "requires": { + "boolbase": "~1.0.0" + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + }, + "svgo": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/svgo/-/svgo-1.3.2.tgz", + "integrity": "sha512-yhy/sQYxR5BkC98CY7o31VGsg014AKLEPxdfhora76l36hD9Rdy5NZA/Ocn6yayNPgSamYdtX2rFJdcv07AYVw==", + "requires": { + "chalk": "^2.4.1", + "coa": "^2.0.2", + "css-select": "^2.0.0", + "css-select-base-adapter": "^0.1.1", + "css-tree": "1.0.0-alpha.37", + "csso": "^4.0.2", + "js-yaml": "^3.13.1", + "mkdirp": "~0.5.1", + "object.values": "^1.1.0", + "sax": "~1.2.4", + "stable": "^0.1.8", + "unquote": "~1.1.1", + "util.promisify": "~1.0.0" + } + }, + "uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==" + } + } + }, + "gatsby-plugin-typescript": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-typescript/-/gatsby-plugin-typescript-4.14.0.tgz", + "integrity": "sha512-iAeC1dnpj99hjnRpD4FetXaJ9b321AuIf0q9vAw4G9FvddG0pxDtg3X9roUV8cmJ+VaLNsLr0DYc4fvOfrFGUQ==", + "requires": { + "@babel/core": "^7.15.5", + "@babel/plugin-proposal-nullish-coalescing-operator": "^7.14.5", + "@babel/plugin-proposal-numeric-separator": "^7.14.5", + "@babel/plugin-proposal-optional-chaining": "^7.14.5", + "@babel/preset-typescript": "^7.15.0", + "@babel/runtime": "^7.15.4", + "babel-plugin-remove-graphql-queries": "^4.14.0" + } + }, + "gatsby-plugin-utils": { + "version": "3.8.0", + "resolved": "https://registry.npmjs.org/gatsby-plugin-utils/-/gatsby-plugin-utils-3.8.0.tgz", + "integrity": "sha512-dLFk+4E2BJrSuPz5/cLUyw4/dDbyMtruLww2XnFk34DVxg16FHIBYcY7p5IbfmDiBmMtlgJFqxBHj1zt8l6syw==", + "requires": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "gatsby-sharp": "^0.8.0", + "graphql-compose": "^9.0.7", + "import-from": "^4.0.0", + "joi": "^17.4.2", + "mime": "^3.0.0", + "mini-svg-data-uri": "^1.4.4", + "svgo": "^2.8.0" + }, + "dependencies": { + "mime": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-3.0.0.tgz", + "integrity": "sha512-jSCU7/VB1loIWBZe14aEYHU/+1UMEHoaO7qxCOVJOw9GgH72VAWppxNcjU+x9a2k3GSIBXNKxXQFqRvvZ7vr3A==" + } + } + }, + "gatsby-react-router-scroll": { + "version": "5.14.0", + "resolved": "https://registry.npmjs.org/gatsby-react-router-scroll/-/gatsby-react-router-scroll-5.14.0.tgz", + "integrity": "sha512-jyqAmmo2UK6v/qRfx8bqlRkjiSYtJRUWNb4nx3bpEIvMlN/vGdJtJ60LsGkRJ5g6U6MybfVX7kUFjgjZdgtqHA==", + "requires": { + "@babel/runtime": "^7.15.4", + "prop-types": "^15.8.1" + } + }, + "gatsby-sharp": { + "version": "0.8.0", + "resolved": "https://registry.npmjs.org/gatsby-sharp/-/gatsby-sharp-0.8.0.tgz", + "integrity": "sha512-As590vHGlCiN9iCWneJo/pJYZjkWykjaFvoKAwPcv6Twn3+6l7ExKOBe9v/WmigALU23dI6vWP0JYvXmmvNYBg==", + "requires": { + "@types/sharp": "^0.30.0", + "sharp": "^0.30.3" + } + }, + "gatsby-source-filesystem": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-source-filesystem/-/gatsby-source-filesystem-4.14.0.tgz", + "integrity": "sha512-r6QTNS0Y8DAWN/cHtA3P/cRXE41TAa0QuOeK+6lnoxAbDmgGt2/EvfXy0wIDvKJTYLQQL0WgY5/pxbvsUqTS5g==", + "requires": { + "@babel/runtime": "^7.15.4", + "chokidar": "^3.5.2", + "file-type": "^16.5.3", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "got": "^9.6.0", + "md5-file": "^5.0.0", + "mime": "^2.5.2", + "pretty-bytes": "^5.4.1", + "progress": "^2.0.3", + "valid-url": "^1.0.9", + "xstate": "^4.26.1" + }, + "dependencies": { + "@sindresorhus/is": { + "version": "0.14.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-0.14.0.tgz", + "integrity": "sha512-9NET910DNaIPngYnLLPeg+Ogzqsi9uM4mSboU5y6p8S5DzMTVEsJZrawi+BoDNUVBa2DhJqQYUFvMDfgU062LQ==" + }, + "@szmarczak/http-timer": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-1.1.2.tgz", + "integrity": "sha512-XIB2XbzHTN6ieIjfIMV9hlVcfPU26s2vafYWQcZHWXHOxiaRZYEDKEwdl129Zyg50+foYV2jCgtrqSA6qNuNSA==", + "requires": { + "defer-to-connect": "^1.0.1" + } + }, + "cacheable-request": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-6.1.0.tgz", + "integrity": "sha512-Oj3cAGPCqOZX7Rz64Uny2GYAZNliQSqfbePrgAQ1wKAihYmCUnraBtJtKcGR4xz7wF+LoJC+ssFZvv5BgF9Igg==", + "requires": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^3.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^4.1.0", + "responselike": "^1.0.2" + }, + "dependencies": { + "get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "requires": { + "pump": "^3.0.0" + } + }, + "lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==" + } + } + }, + "decompress-response": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-3.3.0.tgz", + "integrity": "sha1-gKTdMjdIOEv6JICDYirt7Jgq3/M=", + "requires": { + "mimic-response": "^1.0.0" + } + }, + "defer-to-connect": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-1.1.3.tgz", + "integrity": "sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ==" + }, + "get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "requires": { + "pump": "^3.0.0" + } + }, + "got": { + "version": "9.6.0", + "resolved": "https://registry.npmjs.org/got/-/got-9.6.0.tgz", + "integrity": "sha512-R7eWptXuGYxwijs0eV+v3o6+XH1IqVK8dJOEecQfTmkncw9AV4dcw/Dhxi8MdlqPthxxpZyizMzyg8RTmEsG+Q==", + "requires": { + "@sindresorhus/is": "^0.14.0", + "@szmarczak/http-timer": "^1.1.2", + "cacheable-request": "^6.0.0", + "decompress-response": "^3.3.0", + "duplexer3": "^0.1.4", + "get-stream": "^4.1.0", + "lowercase-keys": "^1.0.1", + "mimic-response": "^1.0.1", + "p-cancelable": "^1.0.0", + "to-readable-stream": "^1.0.0", + "url-parse-lax": "^3.0.0" + } + }, + "json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha1-Wx85evx11ne96Lz8Dkfh+aPZqJg=" + }, + "keyv": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-3.1.0.tgz", + "integrity": "sha512-9ykJ/46SN/9KPM/sichzQ7OvXyGDYKGTaDlKMGCAlg2UK8KRy4jb0d8sFc+0Tt0YYnThq8X2RZgCg74RPxgcVA==", + "requires": { + "json-buffer": "3.0.0" + } + }, + "lowercase-keys": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-1.0.1.tgz", + "integrity": "sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA==" + }, + "normalize-url": { + "version": "4.5.1", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-4.5.1.tgz", + "integrity": "sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA==" + }, + "p-cancelable": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-1.1.0.tgz", + "integrity": "sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw==" + }, + "responselike": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-1.0.2.tgz", + "integrity": "sha1-kYcg7ztjHFZCvgaPFa3lpG9Loec=", + "requires": { + "lowercase-keys": "^1.0.0" + } + } + } + }, + "gatsby-telemetry": { + "version": "3.14.0", + "resolved": "https://registry.npmjs.org/gatsby-telemetry/-/gatsby-telemetry-3.14.0.tgz", + "integrity": "sha512-QnlN3nvb+1gYsY6cIQKAuvkhx9uoOg71yuEYB0EFQdgcnyIbWlBVRHId8wOXoQHwRYFmatvxBmcKlVF8FCs61A==", + "requires": { + "@babel/code-frame": "^7.14.0", + "@babel/runtime": "^7.15.4", + "@turist/fetch": "^7.1.7", + "@turist/time": "^0.0.2", + "async-retry-ng": "^2.0.1", + "boxen": "^4.2.0", + "configstore": "^5.0.1", + "fs-extra": "^10.1.0", + "gatsby-core-utils": "^3.14.0", + "git-up": "^4.0.5", + "is-docker": "^2.2.1", + "lodash": "^4.17.21", + "node-fetch": "^2.6.7" + }, + "dependencies": { + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "requires": { + "color-convert": "^2.0.1" + } + }, + "boxen": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/boxen/-/boxen-4.2.0.tgz", + "integrity": "sha512-eB4uT9RGzg2odpER62bBwSLvUeGC+WbRjjyyFhGsKnc8wp/m0+hQsMUvUe3H2V0D5vw0nBdO1hCJoZo5mKeuIQ==", + "requires": { + "ansi-align": "^3.0.0", + "camelcase": "^5.3.1", + "chalk": "^3.0.0", + "cli-boxes": "^2.2.0", + "string-width": "^4.1.0", + "term-size": "^2.1.0", + "type-fest": "^0.8.1", + "widest-line": "^3.1.0" + } + }, + "chalk": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-3.0.0.tgz", + "integrity": "sha512-4D3B6Wf41KOYRFdszmDqMCGq5VV/uMAB273JILmO+3jAlh8X4qDtdtgCR3fxtbLEMzSx22QdhnDcJvu2u1fVwg==", + "requires": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + } + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + }, + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==" + }, + "supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "requires": { + "has-flag": "^4.0.0" + } + }, + "type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==" + } + } + }, + "gatsby-transformer-remark": { + "version": "5.14.0", + "resolved": "https://registry.npmmirror.com/gatsby-transformer-remark/-/gatsby-transformer-remark-5.14.0.tgz", + "integrity": "sha512-QHGnNRx9M+YMZBiS3dqHCvmSLgpGAXo/WRl8hNtCyO3MfF1oZlbAYkUWtMwNuwIWAtGmf/SifxzzKhiWRgfNjA==", + "requires": { + "@babel/runtime": "^7.15.4", + "gatsby-core-utils": "^3.14.0", + "gray-matter": "^4.0.3", + "hast-util-raw": "^6.0.2", + "hast-util-to-html": "^7.1.3", + "lodash": "^4.17.21", + "mdast-util-to-hast": "^10.2.0", + "mdast-util-to-string": "^2.0.0", + "mdast-util-toc": "^5.1.0", + "remark": "^13.0.0", + "remark-footnotes": "^3.0.0", + "remark-gfm": "^1.0.0", + "remark-parse": "^9.0.0", + "remark-retext": "^4.0.0", + "remark-stringify": "^9.0.1", + "retext-english": "^3.0.4", + "sanitize-html": "^1.27.5", + "underscore.string": "^3.3.6", + "unified": "^9.2.2", + "unist-util-remove-position": "^3.0.0", + "unist-util-select": "^3.0.4", + "unist-util-visit": "^2.0.3" + } + }, + "gatsby-transformer-sharp": { + "version": "4.14.0", + "resolved": "https://registry.npmjs.org/gatsby-transformer-sharp/-/gatsby-transformer-sharp-4.14.0.tgz", + "integrity": "sha512-V8tpCavnmK05GK7LoSAUyd/KZBoZr2NK4IdyIbBdxOcXBqtsD/yrThOyLP00hEV03myTv/6nzSIBZdEL+AGG5g==", + "requires": { + "@babel/runtime": "^7.15.4", + "@gatsbyjs/potrace": "^2.2.0", + "bluebird": "^3.7.2", + "common-tags": "^1.8.2", + "fs-extra": "^10.1.0", + "probe-image-size": "^7.2.3", + "semver": "^7.3.7", + "sharp": "^0.30.3" + } + }, + "gatsby-worker": { + "version": "1.14.0", + "resolved": "https://registry.npmjs.org/gatsby-worker/-/gatsby-worker-1.14.0.tgz", + "integrity": "sha512-Zxa295xBIdgsjg0evBFetm8ctkzi7l1cbPJ8VR5440SV8Mun1d1iPJYl070UazNSYz7UK1lTf1B0ISJYUg31VQ==", + "requires": { + "@babel/core": "^7.15.5", + "@babel/runtime": "^7.15.4" + } + }, + "gauge": { + "version": "2.7.4", + "resolved": "https://registry.npmjs.org/gauge/-/gauge-2.7.4.tgz", + "integrity": "sha1-LANAXHU4w51+s3sxcCLjJfsBi/c=", + "requires": { + "aproba": "^1.0.3", + "console-control-strings": "^1.0.0", + "has-unicode": "^2.0.0", + "object-assign": "^4.1.0", + "signal-exit": "^3.0.0", + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1", + "wide-align": "^1.1.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=" + }, + "is-fullwidth-code-point": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz", + "integrity": "sha1-754xOG8DGn8NZDr4L95QxFfvAMs=", + "requires": { + "number-is-nan": "^1.0.0" + } + }, + "string-width": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-1.0.2.tgz", + "integrity": "sha1-EYvfW4zcUaKn5w0hHgfisLmxB9M=", + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + } + }, + "strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", + "requires": { + "ansi-regex": "^2.0.0" + } + } + } + }, + "gaze": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/gaze/-/gaze-1.1.3.tgz", + "integrity": "sha512-BRdNm8hbWzFzWHERTrejLqwHDfS4GibPoq5wjTPIoJHoBtKGPg3xAFfxmM+9ztbXelxcf2hwQcaz1PtmFeue8g==", + "requires": { + "globule": "^1.0.0" + } + }, + "gensync": { + "version": "1.0.0-beta.2", + "resolved": "https://registry.npmjs.org/gensync/-/gensync-1.0.0-beta.2.tgz", + "integrity": "sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg==" + }, + "get-caller-file": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", + "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==" + }, + "get-intrinsic": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/get-intrinsic/-/get-intrinsic-1.1.1.tgz", + "integrity": "sha512-kWZrnVM42QCiEA2Ig1bG8zjoIMOgxWwYCEeNdwY6Tv/cOSeGpcoX4pXHfKUxNKVoArnrEr2e9srnAxxGIraS9Q==", + "requires": { + "function-bind": "^1.1.1", + "has": "^1.0.3", + "has-symbols": "^1.0.1" + } + }, + "get-own-enumerable-property-symbols": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/get-own-enumerable-property-symbols/-/get-own-enumerable-property-symbols-3.0.2.tgz", + "integrity": "sha512-I0UBV/XOz1XkIJHEUDMZAbzCThU/H8DxmSfmdGcKPnVhu2VfFqr34jr9777IyaTYvxjedWhqVIilEDsCdP5G6g==" + }, + "get-port": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/get-port/-/get-port-3.2.0.tgz", + "integrity": "sha1-3Xzn3hh8Bsi/NTeWrHHgmfCYDrw=" + }, + "get-stdin": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/get-stdin/-/get-stdin-4.0.1.tgz", + "integrity": "sha512-F5aQMywwJ2n85s4hJPTT9RPxGmubonuB10MNYo17/xph174n2MIR33HRguhzVag10O/npM7SPk73LMZNP+FaWw==" + }, + "get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "requires": { + "pump": "^3.0.0" + } + }, + "get-symbol-description": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/get-symbol-description/-/get-symbol-description-1.0.0.tgz", + "integrity": "sha512-2EmdH1YvIQiZpltCNgkuiUnyukzxM/R6NDJX31Ke3BG1Nq5b0S2PhX59UKi9vZpPDQVdqn+1IcaAwnzTT5vCjw==", + "requires": { + "call-bind": "^1.0.2", + "get-intrinsic": "^1.1.1" + } + }, + "getpass": { + "version": "0.1.7", + "resolved": "https://registry.npmmirror.com/getpass/-/getpass-0.1.7.tgz", + "integrity": "sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==", + "requires": { + "assert-plus": "^1.0.0" + } + }, + "gifwrap": { + "version": "0.9.4", + "resolved": "https://registry.npmjs.org/gifwrap/-/gifwrap-0.9.4.tgz", + "integrity": "sha512-MDMwbhASQuVeD4JKd1fKgNgCRL3fGqMM4WaqpNhWO0JiMOAjbQdumbs4BbBZEy9/M00EHEjKN3HieVhCUlwjeQ==", + "requires": { + "image-q": "^4.0.0", + "omggif": "^1.0.10" + } + }, + "git-up": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/git-up/-/git-up-4.0.5.tgz", + "integrity": "sha512-YUvVDg/vX3d0syBsk/CKUTib0srcQME0JyHkL5BaYdwLsiCslPWmDSi8PUMo9pXYjrryMcmsCoCgsTpSCJEQaA==", + "requires": { + "is-ssh": "^1.3.0", + "parse-url": "^6.0.0" + } + }, + "github-from-package": { + "version": "0.0.0", + "resolved": "https://registry.npmjs.org/github-from-package/-/github-from-package-0.0.0.tgz", + "integrity": "sha1-l/tdlr/eiXMxPyDoKI75oWf6ZM4=" + }, + "github-slugger": { + "version": "1.4.0", + "resolved": "https://registry.npmmirror.com/github-slugger/-/github-slugger-1.4.0.tgz", + "integrity": "sha512-w0dzqw/nt51xMVmlaV1+JRzN+oCa1KfcgGEWhxUG16wbdA+Xnt/yoFO8Z8x/V82ZcZ0wy6ln9QDup5avbhiDhQ==" + }, + "glob": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", + "integrity": "sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q==", + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + }, + "glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "requires": { + "is-glob": "^4.0.1" + } + }, + "glob-to-regexp": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz", + "integrity": "sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw==" + }, + "global": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/global/-/global-4.4.0.tgz", + "integrity": "sha512-wv/LAoHdRE3BeTGz53FAamhGlPLhlssK45usmGFThIi4XqnBmjKQ16u+RNbP7WvigRZDxUsM0J3gcQ5yicaL0w==", + "requires": { + "min-document": "^2.19.0", + "process": "^0.11.10" + } + }, + "global-dirs": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/global-dirs/-/global-dirs-3.0.0.tgz", + "integrity": "sha512-v8ho2DS5RiCjftj1nD9NmnfaOzTdud7RRnVd9kFNOjqZbISlx5DQ+OrTkywgd0dIt7oFCvKetZSHoHcP3sDdiA==", + "requires": { + "ini": "2.0.0" + } + }, + "global-modules": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-2.0.0.tgz", + "integrity": "sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A==", + "requires": { + "global-prefix": "^3.0.0" + } + }, + "global-prefix": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-3.0.0.tgz", + "integrity": "sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg==", + "requires": { + "ini": "^1.3.5", + "kind-of": "^6.0.2", + "which": "^1.3.1" + }, + "dependencies": { + "ini": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/ini/-/ini-1.3.8.tgz", + "integrity": "sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==" + } + } + }, + "globals": { + "version": "11.12.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-11.12.0.tgz", + "integrity": "sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==" + }, + "globby": { + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/globby/-/globby-11.1.0.tgz", + "integrity": "sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g==", + "requires": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.2.9", + "ignore": "^5.2.0", + "merge2": "^1.4.1", + "slash": "^3.0.0" + } + }, + "globule": { + "version": "1.3.3", + "resolved": "https://registry.npmmirror.com/globule/-/globule-1.3.3.tgz", + "integrity": "sha512-mb1aYtDbIjTu4ShMB85m3UzjX9BVKe9WCzsnfMSZk+K5GpIbBOexgg4PPCt5eHDEG5/ZQAUX2Kct02zfiPLsKg==", + "requires": { + "glob": "~7.1.1", + "lodash": "~4.17.10", + "minimatch": "~3.0.2" + }, + "dependencies": { + "glob": { + "version": "7.1.7", + "resolved": "https://registry.npmmirror.com/glob/-/glob-7.1.7.tgz", + "integrity": "sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ==", + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + }, + "minimatch": { + "version": "3.0.8", + "resolved": "https://registry.npmmirror.com/minimatch/-/minimatch-3.0.8.tgz", + "integrity": "sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q==", + "requires": { + "brace-expansion": "^1.1.7" + } + } + } + }, + "got": { + "version": "11.8.3", + "resolved": "https://registry.npmjs.org/got/-/got-11.8.3.tgz", + "integrity": "sha512-7gtQ5KiPh1RtGS9/Jbv1ofDpBFuq42gyfEib+ejaRBJuj/3tQFeR5+gw57e4ipaU8c/rCjvX6fkQz2lyDlGAOg==", + "requires": { + "@sindresorhus/is": "^4.0.0", + "@szmarczak/http-timer": "^4.0.5", + "@types/cacheable-request": "^6.0.1", + "@types/responselike": "^1.0.0", + "cacheable-lookup": "^5.0.3", + "cacheable-request": "^7.0.2", + "decompress-response": "^6.0.0", + "http2-wrapper": "^1.0.0-beta.5.2", + "lowercase-keys": "^2.0.0", + "p-cancelable": "^2.0.0", + "responselike": "^2.0.0" + } + }, + "graceful-fs": { + "version": "4.2.10", + "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.10.tgz", + "integrity": "sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA==" + }, + "graphql": { + "version": "15.8.0", + "resolved": "https://registry.npmjs.org/graphql/-/graphql-15.8.0.tgz", + "integrity": "sha512-5gghUc24tP9HRznNpV2+FIoq3xKkj5dTQqf4v0CpdPbFVwFkWoxOM+o+2OC9ZSvjEMTjfmG9QT+gcvggTwW1zw==" + }, + "graphql-compose": { + "version": "9.0.8", + "resolved": "https://registry.npmjs.org/graphql-compose/-/graphql-compose-9.0.8.tgz", + "integrity": "sha512-I3zvygpVz5hOWk2cYL6yhbgfKbNWbiZFNXlWkv/55U+lX6Y3tL+SyY3zunw7QWrN/qtwG2DqZb13SHTv2MgdEQ==", + "requires": { + "graphql-type-json": "0.3.2" + } + }, + "graphql-config": { + "version": "3.4.1", + "resolved": "https://registry.npmjs.org/graphql-config/-/graphql-config-3.4.1.tgz", + "integrity": "sha512-g9WyK4JZl1Ko++FSyE5Ir2g66njfxGzrDDhBOwnkoWf/t3TnnZG6BBkWP+pkqVJ5pqMJGPKHNrbew8jRxStjhw==", + "requires": { + "@endemolshinegroup/cosmiconfig-typescript-loader": "3.0.2", + "@graphql-tools/graphql-file-loader": "^6.0.0", + "@graphql-tools/json-file-loader": "^6.0.0", + "@graphql-tools/load": "^6.0.0", + "@graphql-tools/merge": "6.0.0 - 6.2.14", + "@graphql-tools/url-loader": "^6.0.0", + "@graphql-tools/utils": "^7.0.0", + "cosmiconfig": "7.0.0", + "cosmiconfig-toml-loader": "1.0.0", + "minimatch": "3.0.4", + "string-env-interpolation": "1.0.1" + }, + "dependencies": { + "@graphql-tools/load": { + "version": "6.2.8", + "resolved": "https://registry.npmjs.org/@graphql-tools/load/-/load-6.2.8.tgz", + "integrity": "sha512-JpbyXOXd8fJXdBh2ta0Q4w8ia6uK5FHzrTNmcvYBvflFuWly2LDTk2abbSl81zKkzswQMEd2UIYghXELRg8eTA==", + "requires": { + "@graphql-tools/merge": "^6.2.12", + "@graphql-tools/utils": "^7.5.0", + "globby": "11.0.3", + "import-from": "3.0.0", + "is-glob": "4.0.1", + "p-limit": "3.1.0", + "tslib": "~2.2.0", + "unixify": "1.0.0", + "valid-url": "1.0.9" + } + }, + "@graphql-tools/merge": { + "version": "6.2.14", + "resolved": "https://registry.npmjs.org/@graphql-tools/merge/-/merge-6.2.14.tgz", + "integrity": "sha512-RWT4Td0ROJai2eR66NHejgf8UwnXJqZxXgDWDI+7hua5vNA2OW8Mf9K1Wav1ZkjWnuRp4ztNtkZGie5ISw55ow==", + "requires": { + "@graphql-tools/schema": "^7.0.0", + "@graphql-tools/utils": "^7.7.0", + "tslib": "~2.2.0" + } + }, + "@graphql-tools/schema": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/@graphql-tools/schema/-/schema-7.1.5.tgz", + "integrity": "sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA==", + "requires": { + "@graphql-tools/utils": "^7.1.2", + "tslib": "~2.2.0", + "value-or-promise": "1.0.6" + } + }, + "@graphql-tools/utils": { + "version": "7.10.0", + "resolved": "https://registry.npmjs.org/@graphql-tools/utils/-/utils-7.10.0.tgz", + "integrity": "sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w==", + "requires": { + "@ardatan/aggregate-error": "0.0.6", + "camel-case": "4.1.2", + "tslib": "~2.2.0" + } + }, + "cosmiconfig": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.0.tgz", + "integrity": "sha512-pondGvTuVYDk++upghXJabWzL6Kxu6f26ljFw64Swq9v6sQPUL3EUlVDV56diOjpCayKihL6hVe8exIACU4XcA==", + "requires": { + "@types/parse-json": "^4.0.0", + "import-fresh": "^3.2.1", + "parse-json": "^5.0.0", + "path-type": "^4.0.0", + "yaml": "^1.10.0" + } + }, + "globby": { + "version": "11.0.3", + "resolved": "https://registry.npmjs.org/globby/-/globby-11.0.3.tgz", + "integrity": "sha512-ffdmosjA807y7+lA1NM0jELARVmYul/715xiILEjo3hBLPTcirgQNnXECn5g3mtR8TOLCVbkfua1Hpen25/Xcg==", + "requires": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.1.1", + "ignore": "^5.1.4", + "merge2": "^1.3.0", + "slash": "^3.0.0" + } + }, + "import-from": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/import-from/-/import-from-3.0.0.tgz", + "integrity": "sha512-CiuXOFFSzkU5x/CR0+z7T91Iht4CXgfCxVOFRhh2Zyhg5wOpWvvDLQUsWl+gcN+QscYBjez8hDCt85O7RLDttQ==", + "requires": { + "resolve-from": "^5.0.0" + } + }, + "is-glob": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.1.tgz", + "integrity": "sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg==", + "requires": { + "is-extglob": "^2.1.1" + } + }, + "minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "requires": { + "brace-expansion": "^1.1.7" + } + }, + "p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "requires": { + "yocto-queue": "^0.1.0" + } + }, + "tslib": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.2.0.tgz", + "integrity": "sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w==" + }, + "value-or-promise": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.6.tgz", + "integrity": "sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg==" + } + } + }, + "graphql-playground-html": { + "version": "1.6.30", + "resolved": "https://registry.npmjs.org/graphql-playground-html/-/graphql-playground-html-1.6.30.tgz", + "integrity": "sha512-tpCujhsJMva4aqE8ULnF7/l3xw4sNRZcSHu+R00VV+W0mfp+Q20Plvcrp+5UXD+2yS6oyCXncA+zoQJQqhGCEw==", + "requires": { + "xss": "^1.0.6" + } + }, + "graphql-playground-middleware-express": { + "version": "1.7.23", + "resolved": "https://registry.npmjs.org/graphql-playground-middleware-express/-/graphql-playground-middleware-express-1.7.23.tgz", + "integrity": "sha512-M/zbTyC1rkgiQjFSgmzAv6umMHOphYLNWZp6Ye5QrD77WfGOOoSqDsVmGUczc2pDkEPEzzGB/bvBO5rdzaTRgw==", + "requires": { + "graphql-playground-html": "^1.6.30" + } + }, + "graphql-tag": { + "version": "2.12.6", + "resolved": "https://registry.npmjs.org/graphql-tag/-/graphql-tag-2.12.6.tgz", + "integrity": "sha512-FdSNcu2QQcWnM2VNvSCCDCVS5PpPqpzgFT8+GXzqJuoDd0CBncxCY278u4mhRO7tMgo2JjgJA5aZ+nWSQ/Z+xg==", + "requires": { + "tslib": "^2.1.0" + } + }, + "graphql-type-json": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/graphql-type-json/-/graphql-type-json-0.3.2.tgz", + "integrity": "sha512-J+vjof74oMlCWXSvt0DOf2APEdZOCdubEvGDUAlqH//VBYcOYsGgRW7Xzorr44LvkjiuvecWc8fChxuZZbChtg==", + "requires": {} + }, + "graphql-ws": { + "version": "4.9.0", + "resolved": "https://registry.npmjs.org/graphql-ws/-/graphql-ws-4.9.0.tgz", + "integrity": "sha512-sHkK9+lUm20/BGawNEWNtVAeJzhZeBg21VmvmLoT5NdGVeZWv5PdIhkcayQIAgjSyyQ17WMKmbDijIPG2On+Ag==", + "requires": {} + }, + "gray-matter": { + "version": "4.0.3", + "resolved": "https://registry.npmmirror.com/gray-matter/-/gray-matter-4.0.3.tgz", + "integrity": "sha512-5v6yZd4JK3eMI3FqqCouswVqwugaA9r4dNZB1wwcmrD02QkV5H0y7XBQW8QwQqEaZY1pM9aqORSORhJRdNK44Q==", + "requires": { + "js-yaml": "^3.13.1", + "kind-of": "^6.0.2", + "section-matter": "^1.0.0", + "strip-bom-string": "^1.0.0" + } + }, + "gzip-size": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-6.0.0.tgz", + "integrity": "sha512-ax7ZYomf6jqPTQ4+XCpUGyXKHk5WweS+e05MBO4/y3WJ5RkmPXNKvX+bx1behVILVwr6JSQvZAku021CHPXG3Q==", + "requires": { + "duplexer": "^0.1.2" + } + }, + "har-schema": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/har-schema/-/har-schema-2.0.0.tgz", + "integrity": "sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==" + }, + "har-validator": { + "version": "5.1.5", + "resolved": "https://registry.npmmirror.com/har-validator/-/har-validator-5.1.5.tgz", + "integrity": "sha512-nmT2T0lljbxdQZfspsno9hgrG3Uir6Ks5afism62poxqBM6sDnMEuPmzTq8XN0OEwqKLLdh1jQI3qyE66Nzb3w==", + "requires": { + "ajv": "^6.12.3", + "har-schema": "^2.0.0" + } + }, + "hard-rejection": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/hard-rejection/-/hard-rejection-2.1.0.tgz", + "integrity": "sha512-VIZB+ibDhx7ObhAe7OVtoEbuP4h/MuOTHJ+J8h/eBXotJYl0fBgR72xDFCKgIh22OJZIOVNxBMWuhAr10r8HdA==" + }, + "has": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz", + "integrity": "sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==", + "requires": { + "function-bind": "^1.1.1" + } + }, + "has-ansi": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/has-ansi/-/has-ansi-2.0.0.tgz", + "integrity": "sha512-C8vBJ8DwUCx19vhm7urhTuUsr4/IyP6l4VzNQDv+ryHQObW3TTTp9yB68WpYgRe2bbaGuZ/se74IqFeVnMnLZg==", + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha512-TIGnTpdo+E3+pCyAluZvtED5p5wCqLdezCyhPZzKPcxvFplEt4i+W7OONCKgeZFT3+y5NZZfOOS/Bdcanm1MYA==" + } + } + }, + "has-bigints": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/has-bigints/-/has-bigints-1.0.2.tgz", + "integrity": "sha512-tSvCKtBr9lkF0Ex0aQiP9N+OpV4zi2r/Nee5VkRDbaqv35RLYMzbwQfFSZZH0kR+Rd6302UJZ2p/bJCEoR3VoQ==" + }, + "has-cors": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/has-cors/-/has-cors-1.1.0.tgz", + "integrity": "sha1-XkdHk/fqmEPRu5nCPu9J/xJv/zk=" + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" + }, + "has-property-descriptors": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz", + "integrity": "sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ==", + "requires": { + "get-intrinsic": "^1.1.1" + } + }, + "has-symbols": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.3.tgz", + "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==" + }, + "has-tostringtag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-tostringtag/-/has-tostringtag-1.0.0.tgz", + "integrity": "sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ==", + "requires": { + "has-symbols": "^1.0.2" + } + }, + "has-unicode": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/has-unicode/-/has-unicode-2.0.1.tgz", + "integrity": "sha1-4Ob+aijPUROIVeCG0Wkedx3iqLk=" + }, + "has-yarn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/has-yarn/-/has-yarn-2.1.0.tgz", + "integrity": "sha512-UqBRqi4ju7T+TqGNdqAO0PaSVGsDGJUBQvk9eUWNGRY1CFGDzYhLWoM7JQEemnlvVcv/YEmc2wNW8BC24EnUsw==" + }, + "hasha": { + "version": "5.2.2", + "resolved": "https://registry.npmjs.org/hasha/-/hasha-5.2.2.tgz", + "integrity": "sha512-Hrp5vIK/xr5SkeN2onO32H0MgNZ0f17HRNH39WfL0SYUNOTZ5Lz1TJ8Pajo/87dYGEFlLMm7mIc/k/s6Bvz9HQ==", + "requires": { + "is-stream": "^2.0.0", + "type-fest": "^0.8.0" + }, + "dependencies": { + "type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==" + } + } + }, + "hast-to-hyperscript": { + "version": "9.0.1", + "resolved": "https://registry.npmmirror.com/hast-to-hyperscript/-/hast-to-hyperscript-9.0.1.tgz", + "integrity": "sha512-zQgLKqF+O2F72S1aa4y2ivxzSlko3MAvxkwG8ehGmNiqd98BIN3JM1rAJPmplEyLmGLO2QZYJtIneOSZ2YbJuA==", + "requires": { + "@types/unist": "^2.0.3", + "comma-separated-tokens": "^1.0.0", + "property-information": "^5.3.0", + "space-separated-tokens": "^1.0.0", + "style-to-object": "^0.3.0", + "unist-util-is": "^4.0.0", + "web-namespaces": "^1.0.0" + } + }, + "hast-util-from-parse5": { + "version": "6.0.1", + "resolved": "https://registry.npmmirror.com/hast-util-from-parse5/-/hast-util-from-parse5-6.0.1.tgz", + "integrity": "sha512-jeJUWiN5pSxW12Rh01smtVkZgZr33wBokLzKLwinYOUfSzm1Nl/c3GUGebDyOKjdsRgMvoVbV0VpAcpjF4NrJA==", + "requires": { + "@types/parse5": "^5.0.0", + "hastscript": "^6.0.0", + "property-information": "^5.0.0", + "vfile": "^4.0.0", + "vfile-location": "^3.2.0", + "web-namespaces": "^1.0.0" + } + }, + "hast-util-is-element": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/hast-util-is-element/-/hast-util-is-element-1.1.0.tgz", + "integrity": "sha512-oUmNua0bFbdrD/ELDSSEadRVtWZOf3iF6Lbv81naqsIV99RnSCieTbWuWCY8BAeEfKJTKl0gRdokv+dELutHGQ==" + }, + "hast-util-parse-selector": { + "version": "2.2.5", + "resolved": "https://registry.npmmirror.com/hast-util-parse-selector/-/hast-util-parse-selector-2.2.5.tgz", + "integrity": "sha512-7j6mrk/qqkSehsM92wQjdIgWM2/BW61u/53G6xmC8i1OmEdKLHbk419QKQUjz6LglWsfqoiHmyMRkP1BGjecNQ==" + }, + "hast-util-raw": { + "version": "6.1.0", + "resolved": "https://registry.npmmirror.com/hast-util-raw/-/hast-util-raw-6.1.0.tgz", + "integrity": "sha512-5FoZLDHBpka20OlZZ4I/+RBw5piVQ8iI1doEvffQhx5CbCyTtP8UCq8Tw6NmTAMtXgsQxmhW7Ly8OdFre5/YMQ==", + "requires": { + "@types/hast": "^2.0.0", + "hast-util-from-parse5": "^6.0.0", + "hast-util-to-parse5": "^6.0.0", + "html-void-elements": "^1.0.0", + "parse5": "^6.0.0", + "unist-util-position": "^3.0.0", + "unist-util-visit": "^2.0.0", + "vfile": "^4.0.0", + "web-namespaces": "^1.0.0", + "xtend": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "hast-util-to-html": { + "version": "7.1.3", + "resolved": "https://registry.npmmirror.com/hast-util-to-html/-/hast-util-to-html-7.1.3.tgz", + "integrity": "sha512-yk2+1p3EJTEE9ZEUkgHsUSVhIpCsL/bvT8E5GzmWc+N1Po5gBw+0F8bo7dpxXR0nu0bQVxVZGX2lBGF21CmeDw==", + "requires": { + "ccount": "^1.0.0", + "comma-separated-tokens": "^1.0.0", + "hast-util-is-element": "^1.0.0", + "hast-util-whitespace": "^1.0.0", + "html-void-elements": "^1.0.0", + "property-information": "^5.0.0", + "space-separated-tokens": "^1.0.0", + "stringify-entities": "^3.0.1", + "unist-util-is": "^4.0.0", + "xtend": "^4.0.0" + } + }, + "hast-util-to-parse5": { + "version": "6.0.0", + "resolved": "https://registry.npmmirror.com/hast-util-to-parse5/-/hast-util-to-parse5-6.0.0.tgz", + "integrity": "sha512-Lu5m6Lgm/fWuz8eWnrKezHtVY83JeRGaNQ2kn9aJgqaxvVkFCZQBEhgodZUDUvoodgyROHDb3r5IxAEdl6suJQ==", + "requires": { + "hast-to-hyperscript": "^9.0.0", + "property-information": "^5.0.0", + "web-namespaces": "^1.0.0", + "xtend": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "hast-util-whitespace": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/hast-util-whitespace/-/hast-util-whitespace-1.0.4.tgz", + "integrity": "sha512-I5GTdSfhYfAPNztx2xJRQpG8cuDSNt599/7YUn7Gx/WxNMsG+a835k97TDkFgk123cwjfwINaZknkKkphx/f2A==" + }, + "hastscript": { + "version": "6.0.0", + "resolved": "https://registry.npmmirror.com/hastscript/-/hastscript-6.0.0.tgz", + "integrity": "sha512-nDM6bvd7lIqDUiYEiu5Sl/+6ReP0BMk/2f4U/Rooccxkj0P5nm+acM5PrGJ/t5I8qPGiqZSE6hVAwZEdZIvP4w==", + "requires": { + "@types/hast": "^2.0.0", + "comma-separated-tokens": "^1.0.0", + "hast-util-parse-selector": "^2.0.0", + "property-information": "^5.0.0", + "space-separated-tokens": "^1.0.0" + } + }, + "header-case": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/header-case/-/header-case-2.0.4.tgz", + "integrity": "sha512-H/vuk5TEEVZwrR0lp2zed9OCo1uAILMlx0JEMgC26rzyJJ3N1v6XkwHHXJQdR2doSjcGPM6OKPYoJgf0plJ11Q==", + "requires": { + "capital-case": "^1.0.4", + "tslib": "^2.0.3" + } + }, + "hoist-non-react-statics": { + "version": "3.3.2", + "resolved": "https://registry.npmmirror.com/hoist-non-react-statics/-/hoist-non-react-statics-3.3.2.tgz", + "integrity": "sha512-/gGivxi8JPKWNm/W0jSmzcMPpfpPLc3dY/6GxhX2hQ9iGj3aDfklV4ET7NjKpSinLpJ5vafa9iiGIEZg10SfBw==", + "requires": { + "react-is": "^16.7.0" + } + }, + "hosted-git-info": { + "version": "3.0.8", + "resolved": "https://registry.npmjs.org/hosted-git-info/-/hosted-git-info-3.0.8.tgz", + "integrity": "sha512-aXpmwoOhRBrw6X3j0h5RloK4x1OzsxMPyxqIHyNfSe2pypkVTZFpEiRoSipPEPlMrh0HW/XsjkJ5WgnCirpNUw==", + "requires": { + "lru-cache": "^6.0.0" + } + }, + "html-dom-parser": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/html-dom-parser/-/html-dom-parser-1.0.2.tgz", + "integrity": "sha512-Jq4oVkVSn+10ut3fyc2P/Fs1jqTo0l45cP6Q8d2ef/9jfkYwulO0QXmyLI0VUiZrXF4czpGgMEJRa52CQ6Fk8Q==", + "requires": { + "domhandler": "4.2.2", + "htmlparser2": "6.1.0" + }, + "dependencies": { + "domhandler": { + "version": "4.2.2", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-4.2.2.tgz", + "integrity": "sha512-PzE9aBMsdZO8TK4BnuJwH0QT41wgMbRzuZrHUcpYncEjmQazq8QEaBWgLG7ZyC/DAZKEgglpIA6j4Qn/HmxS3w==", + "requires": { + "domelementtype": "^2.2.0" + } + } + } + }, + "html-entities": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/html-entities/-/html-entities-2.3.3.tgz", + "integrity": "sha512-DV5Ln36z34NNTDgnz0EWGBLZENelNAtkiFA4kyNOG2tDI6Mz1uSWiq1wAKdyjnJwyDiDO7Fa2SO1CTxPXL8VxA==" + }, + "html-react-parser": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/html-react-parser/-/html-react-parser-1.3.0.tgz", + "integrity": "sha512-lhpkOFH8pwqEjlNUYCWvjT43/JVCZO9MAZuCS6afT1/VP+bZcNxNUs4AUqiMzH0QPSDHwM/GFNXZNok1KTA4BQ==", + "requires": { + "domhandler": "4.2.2", + "html-dom-parser": "1.0.2", + "react-property": "2.0.0", + "style-to-js": "1.1.0" + }, + "dependencies": { + "domhandler": { + "version": "4.2.2", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-4.2.2.tgz", + "integrity": "sha512-PzE9aBMsdZO8TK4BnuJwH0QT41wgMbRzuZrHUcpYncEjmQazq8QEaBWgLG7ZyC/DAZKEgglpIA6j4Qn/HmxS3w==", + "requires": { + "domelementtype": "^2.2.0" + } + } + } + }, + "html-tokenize": { + "version": "2.0.1", + "resolved": "https://registry.npmmirror.com/html-tokenize/-/html-tokenize-2.0.1.tgz", + "integrity": "sha512-QY6S+hZ0f5m1WT8WffYN+Hg+xm/w5I8XeUcAq/ZYP5wVC8xbKi4Whhru3FtrAebD5EhBW8rmFzkDI6eCAuFe2w==", + "requires": { + "buffer-from": "~0.1.1", + "inherits": "~2.0.1", + "minimist": "~1.2.5", + "readable-stream": "~1.0.27-1", + "through2": "~0.4.1" + }, + "dependencies": { + "buffer-from": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/buffer-from/-/buffer-from-0.1.2.tgz", + "integrity": "sha512-RiWIenusJsmI2KcvqQABB83tLxCByE3upSP8QU3rJDMVFGPWLvPQJt/O1Su9moRWeH7d+Q2HYb68f6+v+tw2vg==" + }, + "isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmmirror.com/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==" + }, + "readable-stream": { + "version": "1.0.34", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.0.34.tgz", + "integrity": "sha512-ok1qVCJuRkNmvebYikljxJA/UEsKwLl2nI1OmaqAu4/UE+h0wKCHok4XkL/gvi39OacXvw59RJUOFUkDib2rHg==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==" + } + } + }, + "html-void-elements": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/html-void-elements/-/html-void-elements-1.0.5.tgz", + "integrity": "sha512-uE/TxKuyNIcx44cIWnjr/rfIATDH7ZaOMmstu0CwhFG1Dunhlp4OC6/NMbhiwoq5BpW0ubi303qnEk/PZj614w==" + }, + "htmlparser2": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/htmlparser2/-/htmlparser2-6.1.0.tgz", + "integrity": "sha512-gyyPk6rgonLFEDGoeRgQNaEUvdJ4ktTmmUh/h2t7s+M8oPpIPxgNACWa+6ESR57kXstwqPiCut0V8NRpcwgU7A==", + "requires": { + "domelementtype": "^2.0.1", + "domhandler": "^4.0.0", + "domutils": "^2.5.2", + "entities": "^2.0.0" + } + }, + "http-cache-semantics": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/http-cache-semantics/-/http-cache-semantics-4.1.0.tgz", + "integrity": "sha512-carPklcUh7ROWRK7Cv27RPtdhYhUsela/ue5/jKzjegVvXDqM2ILE9Q2BGn9JZJh1g87cp56su/FgQSzcWS8cQ==" + }, + "http-errors": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/http-errors/-/http-errors-2.0.0.tgz", + "integrity": "sha512-FtwrG/euBzaEjYeRqOgly7G0qviiXoJWnvEH2Z1plBdXgbyjv34pHTSb9zoeHMyDy33+DWy5Wt9Wo+TURtOYSQ==", + "requires": { + "depd": "2.0.0", + "inherits": "2.0.4", + "setprototypeof": "1.2.0", + "statuses": "2.0.1", + "toidentifier": "1.0.1" + } + }, + "http-proxy": { + "version": "1.18.1", + "resolved": "https://registry.npmjs.org/http-proxy/-/http-proxy-1.18.1.tgz", + "integrity": "sha512-7mz/721AbnJwIVbnaSv1Cz3Am0ZLT/UBwkC92VlxhXv/k/BBQfM2fXElQNC27BVGr0uwUpplYPQM9LnaBMR5NQ==", + "requires": { + "eventemitter3": "^4.0.0", + "follow-redirects": "^1.0.0", + "requires-port": "^1.0.0" + }, + "dependencies": { + "eventemitter3": { + "version": "4.0.7", + "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-4.0.7.tgz", + "integrity": "sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw==" + } + } + }, + "http-signature": { + "version": "1.2.0", + "resolved": "https://registry.npmmirror.com/http-signature/-/http-signature-1.2.0.tgz", + "integrity": "sha512-CAbnr6Rz4CYQkLYUtSNXxQPUH2gK8f3iWexVlsnMeD+GjlsQ0Xsy1cOX+mN3dtxYomRy21CiOzU8Uhw6OwncEQ==", + "requires": { + "assert-plus": "^1.0.0", + "jsprim": "^1.2.2", + "sshpk": "^1.7.0" + } + }, + "http2-wrapper": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/http2-wrapper/-/http2-wrapper-1.0.3.tgz", + "integrity": "sha512-V+23sDMr12Wnz7iTcDeJr3O6AIxlnvT/bmaAAAP/Xda35C90p9599p0F1eHR/N1KILWSoWVAiOMFjBBXaXSMxg==", + "requires": { + "quick-lru": "^5.1.1", + "resolve-alpn": "^1.0.0" + } + }, + "human-signals": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/human-signals/-/human-signals-2.1.0.tgz", + "integrity": "sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw==" + }, + "iconv-lite": { + "version": "0.4.24", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.24.tgz", + "integrity": "sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA==", + "requires": { + "safer-buffer": ">= 2.1.2 < 3" + } + }, + "icss-utils": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/icss-utils/-/icss-utils-5.1.0.tgz", + "integrity": "sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA==", + "requires": {} + }, + "idb-keyval": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/idb-keyval/-/idb-keyval-3.2.0.tgz", + "integrity": "sha512-slx8Q6oywCCSfKgPgL0sEsXtPVnSbTLWpyiDcu6msHOyKOLari1TD1qocXVCft80umnkk3/Qqh3lwoFt8T/BPQ==" + }, + "ieee754": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/ieee754/-/ieee754-1.2.1.tgz", + "integrity": "sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==" + }, + "ignore": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.2.0.tgz", + "integrity": "sha512-CmxgYGiEPCLhfLnpPp1MoRmifwEIOgjcHXxOBjv7mY96c+eWScsOP9c112ZyLdWHi0FxHjI+4uVhKYp/gcdRmQ==" + }, + "image-q": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/image-q/-/image-q-4.0.0.tgz", + "integrity": "sha512-PfJGVgIfKQJuq3s0tTDOKtztksibuUEbJQIYT3by6wctQo+Rdlh7ef4evJ5NCdxY4CfMbvFkocEwbl4BF8RlJw==", + "requires": { + "@types/node": "16.9.1" + }, + "dependencies": { + "@types/node": { + "version": "16.9.1", + "resolved": "https://registry.npmjs.org/@types/node/-/node-16.9.1.tgz", + "integrity": "sha512-QpLcX9ZSsq3YYUUnD3nFDY8H7wctAhQj/TFKL8Ya8v5fMm3CFXxo8zStsLAl780ltoYoo1WvKUVGBQK+1ifr7g==" + } + } + }, + "immer": { + "version": "9.0.12", + "resolved": "https://registry.npmjs.org/immer/-/immer-9.0.12.tgz", + "integrity": "sha512-lk7UNmSbAukB5B6dh9fnh5D0bJTOFKxVg2cyJWTYrWRfhLrLMBquONcUs3aFq507hNoIZEDDh8lb8UtOizSMhA==" + }, + "immutable": { + "version": "3.7.6", + "resolved": "https://registry.npmjs.org/immutable/-/immutable-3.7.6.tgz", + "integrity": "sha1-E7TTyxK++hVIKib+Gy665kAHHks=" + }, + "import-fresh": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/import-fresh/-/import-fresh-3.3.0.tgz", + "integrity": "sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw==", + "requires": { + "parent-module": "^1.0.0", + "resolve-from": "^4.0.0" + }, + "dependencies": { + "resolve-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-4.0.0.tgz", + "integrity": "sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==" + } + } + }, + "import-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/import-from/-/import-from-4.0.0.tgz", + "integrity": "sha512-P9J71vT5nLlDeV8FHs5nNxaLbrpfAV5cF5srvbZfpwpcJoM/xZR3hiv+q+SAnuSmuGbXMWud063iIMx/V/EWZQ==" + }, + "import-lazy": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/import-lazy/-/import-lazy-2.1.0.tgz", + "integrity": "sha1-BWmOPUXIjo1+nZLLBYTnfwlvPkM=" + }, + "imurmurhash": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", + "integrity": "sha1-khi5srkoojixPcT7a21XbyMUU+o=" + }, + "indent-string": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/indent-string/-/indent-string-4.0.0.tgz", + "integrity": "sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==" + }, + "inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=", + "requires": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "inherits": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + }, + "ini": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ini/-/ini-2.0.0.tgz", + "integrity": "sha512-7PnF4oN3CvZF23ADhA5wRaYEQpJ8qygSkbtTXWBeXWXmEVRXK+1ITciHWwHhsjv1TmW0MgacIv6hEi5pX5NQdA==" + }, + "inline-style-parser": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/inline-style-parser/-/inline-style-parser-0.1.1.tgz", + "integrity": "sha512-7NXolsK4CAS5+xvdj5OMMbI962hU/wvwoxk+LWR9Ek9bVtyuuYScDN6eS0rUm6TxApFpw7CX1o4uJzcd4AyD3Q==" + }, + "inquirer": { + "version": "7.3.3", + "resolved": "https://registry.npmjs.org/inquirer/-/inquirer-7.3.3.tgz", + "integrity": "sha512-JG3eIAj5V9CwcGvuOmoo6LB9kbAYT8HXffUl6memuszlwDC/qvFAJw49XJ5NROSFNPxp3iQg1GqkFhaY/CR0IA==", + "requires": { + "ansi-escapes": "^4.2.1", + "chalk": "^4.1.0", + "cli-cursor": "^3.1.0", + "cli-width": "^3.0.0", + "external-editor": "^3.0.3", + "figures": "^3.0.0", + "lodash": "^4.17.19", + "mute-stream": "0.0.8", + "run-async": "^2.4.0", + "rxjs": "^6.6.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0", + "through": "^2.3.6" + }, + "dependencies": { + "ansi-escapes": { + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-4.3.2.tgz", + "integrity": "sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==", + "requires": { + "type-fest": "^0.21.3" + } + }, + "type-fest": { + "version": "0.21.3", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz", + "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==" + } + } + }, + "internal-slot": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/internal-slot/-/internal-slot-1.0.3.tgz", + "integrity": "sha512-O0DB1JC/sPyZl7cIo78n5dR7eUSwwpYPiXRhTzNxZVAMUuB8vlnRFyLxdrVToks6XPLVnFfbzaVd5WLjhgg+vA==", + "requires": { + "get-intrinsic": "^1.1.0", + "has": "^1.0.3", + "side-channel": "^1.0.4" + } + }, + "invariant": { + "version": "2.2.4", + "resolved": "https://registry.npmjs.org/invariant/-/invariant-2.2.4.tgz", + "integrity": "sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA==", + "requires": { + "loose-envify": "^1.0.0" + } + }, + "ipaddr.js": { + "version": "1.9.1", + "resolved": "https://registry.npmjs.org/ipaddr.js/-/ipaddr.js-1.9.1.tgz", + "integrity": "sha512-0KI/607xoxSToH7GjN1FfSbLoU0+btTicjsQSWQlh/hZykN8KpmMf7uYwPW3R+akZ6R/w18ZlXSHBYXiYUPO3g==" + }, + "is-absolute": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-absolute/-/is-absolute-1.0.0.tgz", + "integrity": "sha512-dOWoqflvcydARa360Gvv18DZ/gRuHKi2NU/wU5X1ZFzdYfH29nkiNZsF3mp4OJ3H4yo9Mx8A/uAGNzpzPN3yBA==", + "requires": { + "is-relative": "^1.0.0", + "is-windows": "^1.0.1" + } + }, + "is-absolute-url": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/is-absolute-url/-/is-absolute-url-3.0.3.tgz", + "integrity": "sha512-opmNIX7uFnS96NtPmhWQgQx6/NYFgsUXYMllcfzwWKUMwfo8kku1TvE6hkNcH+Q1ts5cMVrsY7j0bxXQDciu9Q==" + }, + "is-alphabetical": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-alphabetical/-/is-alphabetical-1.0.4.tgz", + "integrity": "sha512-DwzsA04LQ10FHTZuL0/grVDk4rFoVH1pjAToYwBrHSxcrBIGQuXrQMtD5U1b0U2XVgKZCTLLP8u2Qxqhy3l2Vg==" + }, + "is-alphanumerical": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-alphanumerical/-/is-alphanumerical-1.0.4.tgz", + "integrity": "sha512-UzoZUr+XfVz3t3v4KyGEniVL9BDRoQtY7tOyrRybkVNjDFWyo1yhXNGrrBTQxp3ib9BLAWs7k2YKBQsFRkZG9A==", + "requires": { + "is-alphabetical": "^1.0.0", + "is-decimal": "^1.0.0" + } + }, + "is-arrayish": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz", + "integrity": "sha1-d8mYQFJ6qOyxqLppe4BkWnqSap0=" + }, + "is-bigint": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/is-bigint/-/is-bigint-1.0.4.tgz", + "integrity": "sha512-zB9CruMamjym81i2JZ3UMn54PKGsQzsJeo6xvN3HJJ4CAsQNB6iRutp2To77OfCNuoxspsIhzaPoO1zyCEhFOg==", + "requires": { + "has-bigints": "^1.0.1" + } + }, + "is-binary-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", + "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "requires": { + "binary-extensions": "^2.0.0" + } + }, + "is-boolean-object": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/is-boolean-object/-/is-boolean-object-1.1.2.tgz", + "integrity": "sha512-gDYaKHJmnj4aWxyj6YHyXVpdQawtVLHU5cb+eztPGczf6cjuTdwve5ZIEfgXqH4e57An1D1AKf8CZ3kYrQRqYA==", + "requires": { + "call-bind": "^1.0.2", + "has-tostringtag": "^1.0.0" + } + }, + "is-buffer": { + "version": "2.0.5", + "resolved": "https://registry.npmmirror.com/is-buffer/-/is-buffer-2.0.5.tgz", + "integrity": "sha512-i2R6zNFDwgEHJyQUtJEk0XFi1i0dPFn/oqjK3/vPCcDeJvW5NQ83V8QbicfF1SupOaB0h8ntgBC2YiE7dfyctQ==" + }, + "is-callable": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/is-callable/-/is-callable-1.2.4.tgz", + "integrity": "sha512-nsuwtxZfMX67Oryl9LCQ+upnC0Z0BgpwntpS89m1H/TLF0zNfzfLMV/9Wa/6MZsj0acpEjAO0KF1xT6ZdLl95w==" + }, + "is-ci": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/is-ci/-/is-ci-2.0.0.tgz", + "integrity": "sha512-YfJT7rkpQB0updsdHLGWrvhBJfcfzNNawYDNIyQXJz0IViGf75O8EBPKSdvw2rF+LGCsX4FZ8tcr3b19LcZq4w==", + "requires": { + "ci-info": "^2.0.0" + } + }, + "is-core-module": { + "version": "2.9.0", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.9.0.tgz", + "integrity": "sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A==", + "requires": { + "has": "^1.0.3" + } + }, + "is-date-object": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/is-date-object/-/is-date-object-1.0.5.tgz", + "integrity": "sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ==", + "requires": { + "has-tostringtag": "^1.0.0" + } + }, + "is-decimal": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-decimal/-/is-decimal-1.0.4.tgz", + "integrity": "sha512-RGdriMmQQvZ2aqaQq3awNA6dCGtKpiDFcOzrTWrDAT2MiWrKQVPmxLGHl7Y2nNu6led0kEyoX0enY0qXYsv9zw==" + }, + "is-docker": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/is-docker/-/is-docker-2.2.1.tgz", + "integrity": "sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ==" + }, + "is-extendable": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/is-extendable/-/is-extendable-0.1.1.tgz", + "integrity": "sha512-5BMULNob1vgFX6EjQw5izWDxrecWK9AM72rugNr0TFldMOi0fj6Jk+zeKIt0xGj4cEfQIJth4w3OKWOJ4f+AFw==" + }, + "is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=" + }, + "is-fullwidth-code-point": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", + "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==" + }, + "is-function": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-function/-/is-function-1.0.2.tgz", + "integrity": "sha512-lw7DUp0aWXYg+CBCN+JKkcE0Q2RayZnSvnZBlwgxHBQhqt5pZNVy4Ri7H9GmmXkdu7LUthszM+Tor1u/2iBcpQ==" + }, + "is-glob": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", + "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "requires": { + "is-extglob": "^2.1.1" + } + }, + "is-hexadecimal": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/is-hexadecimal/-/is-hexadecimal-1.0.4.tgz", + "integrity": "sha512-gyPJuv83bHMpocVYoqof5VDiZveEoGoFL8m3BXNb2VW8Xs+rz9kqO8LOQ5DH6EsuvilT1ApazU0pyl+ytbPtlw==" + }, + "is-installed-globally": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/is-installed-globally/-/is-installed-globally-0.4.0.tgz", + "integrity": "sha512-iwGqO3J21aaSkC7jWnHP/difazwS7SFeIqxv6wEtLU8Y5KlzFTjyqcSIT0d8s4+dDhKytsk9PJZ2BkS5eZwQRQ==", + "requires": { + "global-dirs": "^3.0.0", + "is-path-inside": "^3.0.2" + } + }, + "is-invalid-path": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/is-invalid-path/-/is-invalid-path-0.1.0.tgz", + "integrity": "sha1-MHqFWzzxqTi0TqcNLGEQYFNxTzQ=", + "requires": { + "is-glob": "^2.0.0" + }, + "dependencies": { + "is-extglob": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-1.0.0.tgz", + "integrity": "sha1-rEaBd8SUNAWgkvyPKXYMb/xiBsA=" + }, + "is-glob": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-2.0.1.tgz", + "integrity": "sha1-0Jb5JqPe1WAPP9/ZEZjLCIjC2GM=", + "requires": { + "is-extglob": "^1.0.0" + } + } + } + }, + "is-lower-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-lower-case/-/is-lower-case-2.0.2.tgz", + "integrity": "sha512-bVcMJy4X5Og6VZfdOZstSexlEy20Sr0k/p/b2IlQJlfdKAQuMpiv5w2Ccxb8sKdRUNAG1PnHVHjFSdRDVS6NlQ==", + "requires": { + "tslib": "^2.0.3" + } + }, + "is-negative-zero": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-negative-zero/-/is-negative-zero-2.0.2.tgz", + "integrity": "sha512-dqJvarLawXsFbNDeJW7zAz8ItJ9cd28YufuuFzh0G8pNHjJMnY08Dv7sYX2uF5UpQOwieAeOExEYAWWfu7ZZUA==" + }, + "is-npm": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/is-npm/-/is-npm-5.0.0.tgz", + "integrity": "sha512-WW/rQLOazUq+ST/bCAVBp/2oMERWLsR7OrKyt052dNDk4DHcDE0/7QSXITlmi+VBcV13DfIbysG3tZJm5RfdBA==" + }, + "is-number": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", + "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==" + }, + "is-number-object": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/is-number-object/-/is-number-object-1.0.7.tgz", + "integrity": "sha512-k1U0IRzLMo7ZlYIfzRu23Oh6MiIFasgpb9X76eqfFZAqwH44UI4KTBvBYIZ1dSL9ZzChTB9ShHfLkR4pdW5krQ==", + "requires": { + "has-tostringtag": "^1.0.0" + } + }, + "is-obj": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/is-obj/-/is-obj-2.0.0.tgz", + "integrity": "sha512-drqDG3cbczxxEJRoOXcOjtdp1J/lyp1mNn0xaznRs8+muBhgQcrnbspox5X5fOw0HnMnbfDzvnEMEtqDEJEo8w==" + }, + "is-path-inside": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-3.0.3.tgz", + "integrity": "sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==" + }, + "is-plain-obj": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-1.1.0.tgz", + "integrity": "sha512-yvkRyxmFKEOQ4pNXCmJG5AEQNlXJS5LaONXo5/cLdTZdWvsZ1ioJEonLGAosKlMWE8lwUy/bJzMjcw8az73+Fg==" + }, + "is-plain-object": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-2.0.4.tgz", + "integrity": "sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og==", + "requires": { + "isobject": "^3.0.1" + } + }, + "is-promise": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/is-promise/-/is-promise-4.0.0.tgz", + "integrity": "sha512-hvpoI6korhJMnej285dSg6nu1+e6uxs7zG3BYAm5byqDsgJNWwxzM6z6iZiAgQR4TJ30JmBTOwqZUw3WlyH3AQ==" + }, + "is-regex": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/is-regex/-/is-regex-1.1.4.tgz", + "integrity": "sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg==", + "requires": { + "call-bind": "^1.0.2", + "has-tostringtag": "^1.0.0" + } + }, + "is-regexp": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-regexp/-/is-regexp-1.0.0.tgz", + "integrity": "sha1-/S2INUXEa6xaYz57mgnof6LLUGk=" + }, + "is-relative": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-relative/-/is-relative-1.0.0.tgz", + "integrity": "sha512-Kw/ReK0iqwKeu0MITLFuj0jbPAmEiOsIwyIXvvbfa6QfmN9pkD1M+8pdk7Rl/dTKbH34/XBFMbgD4iMJhLQbGA==", + "requires": { + "is-unc-path": "^1.0.0" + } + }, + "is-relative-url": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-relative-url/-/is-relative-url-3.0.0.tgz", + "integrity": "sha512-U1iSYRlY2GIMGuZx7gezlB5dp1Kheaym7zKzO1PV06mOihiWTXejLwm4poEJysPyXF+HtK/BEd0DVlcCh30pEA==", + "requires": { + "is-absolute-url": "^3.0.0" + } + }, + "is-root": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-root/-/is-root-2.1.0.tgz", + "integrity": "sha512-AGOriNp96vNBd3HtU+RzFEc75FfR5ymiYv8E553I71SCeXBiMsVDUtdio1OEFvrPyLIQ9tVR5RxXIFe5PUFjMg==" + }, + "is-shared-array-buffer": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-shared-array-buffer/-/is-shared-array-buffer-1.0.2.tgz", + "integrity": "sha512-sqN2UDu1/0y6uvXyStCOzyhAjCSlHceFoMKJW8W9EU9cvic/QdsZ0kEU93HEy3IUEFZIiH/3w+AH/UQbPHNdhA==", + "requires": { + "call-bind": "^1.0.2" + } + }, + "is-ssh": { + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/is-ssh/-/is-ssh-1.3.3.tgz", + "integrity": "sha512-NKzJmQzJfEEma3w5cJNcUMxoXfDjz0Zj0eyCalHn2E6VOwlzjZo0yuO2fcBSf8zhFuVCL/82/r5gRcoi6aEPVQ==", + "requires": { + "protocols": "^1.1.0" + } + }, + "is-stream": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-2.0.1.tgz", + "integrity": "sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg==" + }, + "is-string": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/is-string/-/is-string-1.0.7.tgz", + "integrity": "sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg==", + "requires": { + "has-tostringtag": "^1.0.0" + } + }, + "is-symbol": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/is-symbol/-/is-symbol-1.0.4.tgz", + "integrity": "sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg==", + "requires": { + "has-symbols": "^1.0.2" + } + }, + "is-typedarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz", + "integrity": "sha1-5HnICFjfDBsR3dppQPlgEfzaSpo=" + }, + "is-unc-path": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-unc-path/-/is-unc-path-1.0.0.tgz", + "integrity": "sha512-mrGpVd0fs7WWLfVsStvgF6iEJnbjDFZh9/emhRDcGWTduTfNHd9CHeUwH3gYIjdbwo4On6hunkztwOaAw0yllQ==", + "requires": { + "unc-path-regex": "^0.1.2" + } + }, + "is-upper-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/is-upper-case/-/is-upper-case-2.0.2.tgz", + "integrity": "sha512-44pxmxAvnnAOwBg4tHPnkfvgjPwbc5QIsSstNU+YcJ1ovxVzCWpSGosPJOZh/a1tdl81fbgnLc9LLv+x2ywbPQ==", + "requires": { + "tslib": "^2.0.3" + } + }, + "is-valid-domain": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-valid-domain/-/is-valid-domain-0.1.6.tgz", + "integrity": "sha512-ZKtq737eFkZr71At8NxOFcP9O1K89gW3DkdrGMpp1upr/ueWjj+Weh4l9AI4rN0Gt8W2M1w7jrG2b/Yv83Ljpg==", + "requires": { + "punycode": "^2.1.1" + } + }, + "is-valid-path": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/is-valid-path/-/is-valid-path-0.1.1.tgz", + "integrity": "sha1-EQ+f90w39mPh7HkV60UfLbk6yd8=", + "requires": { + "is-invalid-path": "^0.1.0" + } + }, + "is-weakref": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-weakref/-/is-weakref-1.0.2.tgz", + "integrity": "sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ==", + "requires": { + "call-bind": "^1.0.2" + } + }, + "is-windows": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-windows/-/is-windows-1.0.2.tgz", + "integrity": "sha512-eXK1UInq2bPmjyX6e3VHIzMLobc4J94i4AWn+Hpq3OU5KkrRC96OAcR3PRJ/pGu6m8TRnBHP9dkXQVsT/COVIA==" + }, + "is-wsl": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/is-wsl/-/is-wsl-2.2.0.tgz", + "integrity": "sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww==", + "requires": { + "is-docker": "^2.0.0" + } + }, + "is-yarn-global": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/is-yarn-global/-/is-yarn-global-0.3.0.tgz", + "integrity": "sha512-VjSeb/lHmkoyd8ryPVIKvOCn4D1koMqY+vqyjjUfc3xyKtP4dYOxM44sZrnqQSzSds3xyOrUTLTC9LVCVgLngw==" + }, + "isarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-1.0.0.tgz", + "integrity": "sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE=" + }, + "isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=" + }, + "isobject": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", + "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=" + }, + "isomorphic-ws": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/isomorphic-ws/-/isomorphic-ws-4.0.1.tgz", + "integrity": "sha512-BhBvN2MBpWTaSHdWRb/bwdZJ1WaehQ2L1KngkCkfLUGF0mAWAT1sQUQacEmQ0jXkFw/czDXPNQSL5u2/Krsz1w==", + "requires": {} + }, + "isstream": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/isstream/-/isstream-0.1.2.tgz", + "integrity": "sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==" + }, + "iterall": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/iterall/-/iterall-1.3.0.tgz", + "integrity": "sha512-QZ9qOMdF+QLHxy1QIpUHUU1D5pS2CG2P69LF6L6CPjPYA/XMOmKV3PZpawHoAjHNyB0swdVTRxdYT4tbBbxqwg==" + }, + "jest-worker": { + "version": "26.6.2", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-26.6.2.tgz", + "integrity": "sha512-KWYVV1c4i+jbMpaBC+U++4Va0cp8OisU185o73T1vo99hqi7w8tSJfUXYswwqqrjzwxa6KpRK54WhPvwf5w6PQ==", + "requires": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^7.0.0" + }, + "dependencies": { + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==" + }, + "supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "requires": { + "has-flag": "^4.0.0" + } + } + } + }, + "jimp": { + "version": "0.16.1", + "resolved": "https://registry.npmjs.org/jimp/-/jimp-0.16.1.tgz", + "integrity": "sha512-+EKVxbR36Td7Hfd23wKGIeEyHbxShZDX6L8uJkgVW3ESA9GiTEPK08tG1XI2r/0w5Ch0HyJF5kPqF9K7EmGjaw==", + "requires": { + "@babel/runtime": "^7.7.2", + "@jimp/custom": "^0.16.1", + "@jimp/plugins": "^0.16.1", + "@jimp/types": "^0.16.1", + "regenerator-runtime": "^0.13.3" + } + }, + "joi": { + "version": "17.6.0", + "resolved": "https://registry.npmjs.org/joi/-/joi-17.6.0.tgz", + "integrity": "sha512-OX5dG6DTbcr/kbMFj0KGYxuew69HPcAE3K/sZpEV2nP6e/j/C0HV+HNiBPCASxdx5T7DMoa0s8UeHWMnb6n2zw==", + "requires": { + "@hapi/hoek": "^9.0.0", + "@hapi/topo": "^5.0.0", + "@sideway/address": "^4.1.3", + "@sideway/formula": "^3.0.0", + "@sideway/pinpoint": "^2.0.0" + } + }, + "jpeg-js": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/jpeg-js/-/jpeg-js-0.4.2.tgz", + "integrity": "sha512-+az2gi/hvex7eLTMTlbRLOhH6P6WFdk2ITI8HJsaH2VqYO0I594zXSYEP+tf4FW+8Cy68ScDXoAsQdyQanv3sw==" + }, + "js-base64": { + "version": "2.6.4", + "resolved": "https://registry.npmmirror.com/js-base64/-/js-base64-2.6.4.tgz", + "integrity": "sha512-pZe//GGmwJndub7ZghVHz7vjb2LgC1m8B07Au3eYqeqv9emhESByMXxaEgkUkEqJe87oBbSniGYoQNIBklc7IQ==" + }, + "js-cookie": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/js-cookie/-/js-cookie-3.0.1.tgz", + "integrity": "sha512-+0rgsUXZu4ncpPxRL+lNEptWMOWl9etvPHc/koSRp6MPwpRYAhmk0dUG00J4bxVV3r9uUzfo24wW0knS07SKSw==" + }, + "js-tokens": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/js-tokens/-/js-tokens-4.0.0.tgz", + "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==" + }, + "js-yaml": { + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.14.1.tgz", + "integrity": "sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g==", + "requires": { + "argparse": "^1.0.7", + "esprima": "^4.0.0" + } + }, + "jsbn": { + "version": "0.1.1", + "resolved": "https://registry.npmmirror.com/jsbn/-/jsbn-0.1.1.tgz", + "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + }, + "jsesc": { + "version": "2.5.2", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", + "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==" + }, + "json-buffer": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.1.tgz", + "integrity": "sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ==" + }, + "json-loader": { + "version": "0.5.7", + "resolved": "https://registry.npmjs.org/json-loader/-/json-loader-0.5.7.tgz", + "integrity": "sha512-QLPs8Dj7lnf3e3QYS1zkCo+4ZwqOiF9d/nZnYozTISxXWCfNs9yuky5rJw4/W34s7POaNlbZmQGaB5NiXCbP4w==" + }, + "json-parse-even-better-errors": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz", + "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==" + }, + "json-schema": { + "version": "0.4.0", + "resolved": "https://registry.npmmirror.com/json-schema/-/json-schema-0.4.0.tgz", + "integrity": "sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==" + }, + "json-schema-traverse": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz", + "integrity": "sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg==" + }, + "json-source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/json-source-map/-/json-source-map-0.6.1.tgz", + "integrity": "sha512-1QoztHPsMQqhDq0hlXY5ZqcEdUzxQEIxgFkKl4WUp2pgShObl+9ovi4kRh2TfvAfxAoHOJ9vIMEqk3k4iex7tg==" + }, + "json-stable-stringify-without-jsonify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz", + "integrity": "sha1-nbe1lJatPzz+8wp1FC0tkwrXJlE=" + }, + "json-stringify-safe": { + "version": "5.0.1", + "resolved": "https://registry.npmmirror.com/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz", + "integrity": "sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==" + }, + "json5": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-2.2.1.tgz", + "integrity": "sha512-1hqLFMSrGHRHxav9q9gNjJ5EXznIxGVO09xQRrwplcS8qs28pZ8s8hupZAmqDwZUmVZ2Qb2jnyPOWcDH8m8dlA==" + }, + "jsonfile": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-6.1.0.tgz", + "integrity": "sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ==", + "requires": { + "graceful-fs": "^4.1.6", + "universalify": "^2.0.0" + } + }, + "jsprim": { + "version": "1.4.2", + "resolved": "https://registry.npmmirror.com/jsprim/-/jsprim-1.4.2.tgz", + "integrity": "sha512-P2bSOMAc/ciLz6DzgjVlGJP9+BrJWu5UDGK70C2iweC5QBIeFf0ZXRvGjEj2uYgrY2MkAAhsSWHDWlFtEroZWw==", + "requires": { + "assert-plus": "1.0.0", + "extsprintf": "1.3.0", + "json-schema": "0.4.0", + "verror": "1.10.0" + } + }, + "jsx-ast-utils": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/jsx-ast-utils/-/jsx-ast-utils-3.3.0.tgz", + "integrity": "sha512-XzO9luP6L0xkxwhIJMTJQpZo/eeN60K08jHdexfD569AGxeNug6UketeHXEhROoM8aR7EcUoOQmIhcJQjcuq8Q==", + "requires": { + "array-includes": "^3.1.4", + "object.assign": "^4.1.2" + } + }, + "kebab-hash": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/kebab-hash/-/kebab-hash-0.1.2.tgz", + "integrity": "sha512-BTZpq3xgISmQmAVzkISy4eUutsUA7s4IEFlCwOBJjvSFOwyR7I+fza+tBc/rzYWK/NrmFHjfU1IhO3lu29Ib/w==", + "requires": { + "lodash.kebabcase": "^4.1.1" + } + }, + "keyv": { + "version": "4.2.7", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-4.2.7.tgz", + "integrity": "sha512-HeOstD8SXvtWoQhMMBCelcUuZsiV7T7MwsADtOXT0KuwYP9nCxrSoMDeLXNDTLN3VFSuRp38JzoGbbTboq3QQw==", + "requires": { + "compress-brotli": "^1.3.8", + "json-buffer": "3.0.1" + } + }, + "kind-of": { + "version": "6.0.3", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.3.tgz", + "integrity": "sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw==" + }, + "kleur": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/kleur/-/kleur-3.0.3.tgz", + "integrity": "sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w==" + }, + "klona": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/klona/-/klona-2.0.5.tgz", + "integrity": "sha512-pJiBpiXMbt7dkzXe8Ghj/u4FfXOOa98fPW+bihOJ4SjnoijweJrNThJfd3ifXpXhREjpoF2mZVH1GfS9LV3kHQ==" + }, + "language-subtag-registry": { + "version": "0.3.21", + "resolved": "https://registry.npmjs.org/language-subtag-registry/-/language-subtag-registry-0.3.21.tgz", + "integrity": "sha512-L0IqwlIXjilBVVYKFT37X9Ih11Um5NEl9cbJIuU/SwP/zEEAbBPOnEeeuxVMf45ydWQRDQN3Nqc96OgbH1K+Pg==" + }, + "language-tags": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/language-tags/-/language-tags-1.0.5.tgz", + "integrity": "sha1-0yHbxNowuovzAk4ED6XBRmH5GTo=", + "requires": { + "language-subtag-registry": "~0.3.2" + } + }, + "latest-version": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/latest-version/-/latest-version-5.1.0.tgz", + "integrity": "sha512-weT+r0kTkRQdCdYCNtkMwWXQTMEswKrFBkm4ckQOMVhhqhIMI1UT2hMj+1iigIhgSZm5gTmrRXBNoGUgaTY1xA==", + "requires": { + "package-json": "^6.3.0" + } + }, + "levn": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/levn/-/levn-0.4.1.tgz", + "integrity": "sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ==", + "requires": { + "prelude-ls": "^1.2.1", + "type-check": "~0.4.0" + } + }, + "lilconfig": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/lilconfig/-/lilconfig-2.0.5.tgz", + "integrity": "sha512-xaYmXZtTHPAw5m+xLN8ab9C+3a8YmV3asNSPOATITbtwrfbwaLJj8h66H1WMIpALCkqsIzK3h7oQ+PdX+LQ9Eg==" + }, + "lines-and-columns": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/lines-and-columns/-/lines-and-columns-1.2.4.tgz", + "integrity": "sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==" + }, + "lmdb": { + "version": "2.2.6", + "resolved": "https://registry.npmjs.org/lmdb/-/lmdb-2.2.6.tgz", + "integrity": "sha512-UmQV0oZZcV3EN6rjcAjIiuWcc3MYZGWQ0GUYz46Ron5fuTa/dUow7WSQa6leFkvZIKVUdECBWVw96tckfEzUFQ==", + "requires": { + "msgpackr": "^1.5.4", + "nan": "^2.14.2", + "node-gyp-build": "^4.2.3", + "ordered-binary": "^1.2.4", + "weak-lru-cache": "^1.2.2" + } + }, + "load-bmfont": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/load-bmfont/-/load-bmfont-1.4.1.tgz", + "integrity": "sha512-8UyQoYmdRDy81Brz6aLAUhfZLwr5zV0L3taTQ4hju7m6biuwiWiJXjPhBJxbUQJA8PrkvJ/7Enqmwk2sM14soA==", + "requires": { + "buffer-equal": "0.0.1", + "mime": "^1.3.4", + "parse-bmfont-ascii": "^1.0.3", + "parse-bmfont-binary": "^1.0.5", + "parse-bmfont-xml": "^1.1.4", + "phin": "^2.9.1", + "xhr": "^2.0.1", + "xtend": "^4.0.0" + }, + "dependencies": { + "mime": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-1.6.0.tgz", + "integrity": "sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg==" + } + } + }, + "loader-runner": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/loader-runner/-/loader-runner-4.3.0.tgz", + "integrity": "sha512-3R/1M+yS3j5ou80Me59j7F9IMs4PXs3VqRrm0TU3AbKPxlmpoY1TNscJV/oGJXo8qCatFGTfDbY6W6ipGOYXfg==" + }, + "loader-utils": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.2.tgz", + "integrity": "sha512-TM57VeHptv569d/GKh6TAYdzKblwDNiumOdkFnejjD0XwTH87K90w3O7AiJRqdQoXygvi1VQTJTLGhJl7WqA7A==", + "requires": { + "big.js": "^5.2.2", + "emojis-list": "^3.0.0", + "json5": "^2.1.2" + } + }, + "locate-path": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", + "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", + "requires": { + "p-locate": "^4.1.0" + } + }, + "lock": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/lock/-/lock-1.1.0.tgz", + "integrity": "sha1-UxV0mdFlOxNspmRRBx/KYVcD+lU=" + }, + "lockfile": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/lockfile/-/lockfile-1.0.4.tgz", + "integrity": "sha512-cvbTwETRfsFh4nHsL1eGWapU1XFi5Ot9E85sWAwia7Y7EgB7vfqcZhTKZ+l7hCGxSPoushMv5GKhT5PdLv03WA==", + "requires": { + "signal-exit": "^3.0.2" + } + }, + "lodash": { + "version": "4.17.21", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz", + "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==" + }, + "lodash._reinterpolate": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/lodash._reinterpolate/-/lodash._reinterpolate-3.0.0.tgz", + "integrity": "sha1-DM8tiRZq8Ds2Y8eWU4t1rG4RTZ0=" + }, + "lodash.clonedeep": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.clonedeep/-/lodash.clonedeep-4.5.0.tgz", + "integrity": "sha1-4j8/nE+Pvd6HJSnBBxhXoIblzO8=" + }, + "lodash.debounce": { + "version": "4.0.8", + "resolved": "https://registry.npmjs.org/lodash.debounce/-/lodash.debounce-4.0.8.tgz", + "integrity": "sha1-gteb/zCmfEAF/9XiUVMArZyk168=" + }, + "lodash.deburr": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/lodash.deburr/-/lodash.deburr-4.1.0.tgz", + "integrity": "sha1-3bG7s+8HRYwBd7oH3hRCLLAz/5s=" + }, + "lodash.every": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.every/-/lodash.every-4.6.0.tgz", + "integrity": "sha1-64mYS+vENkJ5uzrvu9HKGb+mxqc=" + }, + "lodash.flatten": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.flatten/-/lodash.flatten-4.4.0.tgz", + "integrity": "sha1-8xwiIlqWMtK7+OSt2+8kCqdlph8=" + }, + "lodash.flattendeep": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.flattendeep/-/lodash.flattendeep-4.4.0.tgz", + "integrity": "sha1-+wMJF/hqMTTlvJvsDWngAT3f7bI=" + }, + "lodash.foreach": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.foreach/-/lodash.foreach-4.5.0.tgz", + "integrity": "sha1-Gmo16s5AEoDH8G3d7DUWWrJ+PlM=" + }, + "lodash.get": { + "version": "4.4.2", + "resolved": "https://registry.npmjs.org/lodash.get/-/lodash.get-4.4.2.tgz", + "integrity": "sha1-LRd/ZS+jHpObRDjVNBSZ36OCXpk=" + }, + "lodash.has": { + "version": "4.5.2", + "resolved": "https://registry.npmjs.org/lodash.has/-/lodash.has-4.5.2.tgz", + "integrity": "sha1-0Z9NwQlQWMzL4rDN9O4P5Ko3yGI=" + }, + "lodash.kebabcase": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/lodash.kebabcase/-/lodash.kebabcase-4.1.1.tgz", + "integrity": "sha1-hImxyw0p/4gZXM7KRI/21swpXDY=" + }, + "lodash.map": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.map/-/lodash.map-4.6.0.tgz", + "integrity": "sha1-dx7Hg540c9nEzeKLGTlMNWL09tM=" + }, + "lodash.maxby": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/lodash.maxby/-/lodash.maxby-4.6.0.tgz", + "integrity": "sha1-CCJABo88eiJ6oAqDgOTzjPB4bj0=" + }, + "lodash.memoize": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", + "integrity": "sha1-vMbEmkKihA7Zl/Mj6tpezRguC/4=" + }, + "lodash.merge": { + "version": "4.6.2", + "resolved": "https://registry.npmjs.org/lodash.merge/-/lodash.merge-4.6.2.tgz", + "integrity": "sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ==" + }, + "lodash.sortby": { + "version": "4.7.0", + "resolved": "https://registry.npmjs.org/lodash.sortby/-/lodash.sortby-4.7.0.tgz", + "integrity": "sha1-7dFMgk4sycHgsKG0K7UhBRakJDg=" + }, + "lodash.template": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.template/-/lodash.template-4.5.0.tgz", + "integrity": "sha512-84vYFxIkmidUiFxidA/KjjH9pAycqW+h980j7Fuz5qxRtO9pgB7MDFTdys1N7A5mcucRiDyEq4fusljItR1T/A==", + "requires": { + "lodash._reinterpolate": "^3.0.0", + "lodash.templatesettings": "^4.0.0" + } + }, + "lodash.templatesettings": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/lodash.templatesettings/-/lodash.templatesettings-4.2.0.tgz", + "integrity": "sha512-stgLz+i3Aa9mZgnjr/O+v9ruKZsPsndy7qPZOchbqk2cnTU1ZaldKK+v7m54WoKIyxiuMZTKT2H81F8BeAc3ZQ==", + "requires": { + "lodash._reinterpolate": "^3.0.0" + } + }, + "lodash.truncate": { + "version": "4.4.2", + "resolved": "https://registry.npmjs.org/lodash.truncate/-/lodash.truncate-4.4.2.tgz", + "integrity": "sha1-WjUNoLERO4N+z//VgSy+WNbq4ZM=" + }, + "lodash.uniq": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.uniq/-/lodash.uniq-4.5.0.tgz", + "integrity": "sha1-0CJTc662Uq3BvILklFM5qEJ1R3M=" + }, + "lodash.without": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/lodash.without/-/lodash.without-4.4.0.tgz", + "integrity": "sha1-PNRXSgC2e643OpS3SHcmQFB7eqw=" + }, + "longest-streak": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/longest-streak/-/longest-streak-2.0.4.tgz", + "integrity": "sha512-vM6rUVCVUJJt33bnmHiZEvr7wPT78ztX7rojL+LW51bHtLh6HTjx84LA5W4+oa6aKEJA7jJu5LR6vQRBpA5DVg==" + }, + "loose-envify": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/loose-envify/-/loose-envify-1.4.0.tgz", + "integrity": "sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==", + "requires": { + "js-tokens": "^3.0.0 || ^4.0.0" + } + }, + "lower-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/lower-case/-/lower-case-2.0.2.tgz", + "integrity": "sha512-7fm3l3NAF9WfN6W3JOmf5drwpVqX78JtoGJ3A6W0a6ZnldM41w2fV5D490psKFTpMds8TJse/eHLFFsNHHjHgg==", + "requires": { + "tslib": "^2.0.3" + } + }, + "lower-case-first": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/lower-case-first/-/lower-case-first-2.0.2.tgz", + "integrity": "sha512-EVm/rR94FJTZi3zefZ82fLWab+GX14LJN4HrWBcuo6Evmsl9hEfnqxgcHCKb9q+mNf6EVdsjx/qucYFIIB84pg==", + "requires": { + "tslib": "^2.0.3" + } + }, + "lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==" + }, + "lru-cache": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", + "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", + "requires": { + "yallist": "^4.0.0" + } + }, + "lru-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/lru-queue/-/lru-queue-0.1.0.tgz", + "integrity": "sha1-Jzi9nw089PhEkMVzbEhpmsYyzaM=", + "requires": { + "es5-ext": "~0.10.2" + } + }, + "make-dir": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-3.1.0.tgz", + "integrity": "sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw==", + "requires": { + "semver": "^6.0.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "make-error": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/make-error/-/make-error-1.3.6.tgz", + "integrity": "sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw==" + }, + "map-age-cleaner": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/map-age-cleaner/-/map-age-cleaner-0.1.3.tgz", + "integrity": "sha512-bJzx6nMoP6PDLPBFmg7+xRKeFZvFboMrGlxmNj9ClvX53KrmvM5bXFXEWjbz4cz1AFn+jWJ9z/DJSz7hrs0w3w==", + "requires": { + "p-defer": "^1.0.0" + }, + "dependencies": { + "p-defer": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-defer/-/p-defer-1.0.0.tgz", + "integrity": "sha1-n26xgvbJqozXQwBKfU+WsZaw+ww=" + } + } + }, + "map-cache": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/map-cache/-/map-cache-0.2.2.tgz", + "integrity": "sha1-wyq9C9ZSXZsFFkW7TyasXcmKDb8=" + }, + "map-obj": { + "version": "4.3.0", + "resolved": "https://registry.npmmirror.com/map-obj/-/map-obj-4.3.0.tgz", + "integrity": "sha512-hdN1wVrZbb29eBGiGjJbeP8JbKjq1urkHJ/LIP/NY48MZ1QVXUsQBV1G1zvYFHn1XE06cwjBsOI2K3Ulnj1YXQ==" + }, + "markdown-table": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/markdown-table/-/markdown-table-2.0.0.tgz", + "integrity": "sha512-Ezda85ToJUBhM6WGaG6veasyym+Tbs3cMAw/ZhOPqXiYsr0jgocBV3j3nx+4lk47plLlIqjwuTm/ywVI+zjJ/A==", + "requires": { + "repeat-string": "^1.0.0" + } + }, + "md5-file": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/md5-file/-/md5-file-5.0.0.tgz", + "integrity": "sha512-xbEFXCYVWrSx/gEKS1VPlg84h/4L20znVIulKw6kMfmBUAZNAnF00eczz9ICMl+/hjQGo5KSXRxbL/47X3rmMw==" + }, + "mdast-util-definitions": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/mdast-util-definitions/-/mdast-util-definitions-4.0.0.tgz", + "integrity": "sha512-k8AJ6aNnUkB7IE+5azR9h81O5EQ/cTDXtWdMq9Kk5KcEW/8ritU5CeLg/9HhOC++nALHBlaogJ5jz0Ybk3kPMQ==", + "requires": { + "unist-util-visit": "^2.0.0" + } + }, + "mdast-util-find-and-replace": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/mdast-util-find-and-replace/-/mdast-util-find-and-replace-1.1.1.tgz", + "integrity": "sha512-9cKl33Y21lyckGzpSmEQnIDjEfeeWelN5s1kUW1LwdB0Fkuq2u+4GdqcGEygYxJE8GVqCl0741bYXHgamfWAZA==", + "requires": { + "escape-string-regexp": "^4.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit-parents": "^3.0.0" + }, + "dependencies": { + "escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==" + } + } + }, + "mdast-util-footnote": { + "version": "0.1.7", + "resolved": "https://registry.npmmirror.com/mdast-util-footnote/-/mdast-util-footnote-0.1.7.tgz", + "integrity": "sha512-QxNdO8qSxqbO2e3m09KwDKfWiLgqyCurdWTQ198NpbZ2hxntdc+VKS4fDJCmNWbAroUdYnSthu+XbZ8ovh8C3w==", + "requires": { + "mdast-util-to-markdown": "^0.6.0", + "micromark": "~2.11.0" + } + }, + "mdast-util-from-markdown": { + "version": "0.8.5", + "resolved": "https://registry.npmmirror.com/mdast-util-from-markdown/-/mdast-util-from-markdown-0.8.5.tgz", + "integrity": "sha512-2hkTXtYYnr+NubD/g6KGBS/0mFmBcifAsI0yIWRiRo0PjVs6SSOSOdtzbp6kSGnShDN6G5aWZpKQ2lWRy27mWQ==", + "requires": { + "@types/mdast": "^3.0.0", + "mdast-util-to-string": "^2.0.0", + "micromark": "~2.11.0", + "parse-entities": "^2.0.0", + "unist-util-stringify-position": "^2.0.0" + } + }, + "mdast-util-gfm": { + "version": "0.1.2", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm/-/mdast-util-gfm-0.1.2.tgz", + "integrity": "sha512-NNkhDx/qYcuOWB7xHUGWZYVXvjPFFd6afg6/e2g+SV4r9q5XUcCbV4Wfa3DLYIiD+xAEZc6K4MGaE/m0KDcPwQ==", + "requires": { + "mdast-util-gfm-autolink-literal": "^0.1.0", + "mdast-util-gfm-strikethrough": "^0.2.0", + "mdast-util-gfm-table": "^0.1.0", + "mdast-util-gfm-task-list-item": "^0.1.0", + "mdast-util-to-markdown": "^0.6.1" + } + }, + "mdast-util-gfm-autolink-literal": { + "version": "0.1.3", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-autolink-literal/-/mdast-util-gfm-autolink-literal-0.1.3.tgz", + "integrity": "sha512-GjmLjWrXg1wqMIO9+ZsRik/s7PLwTaeCHVB7vRxUwLntZc8mzmTsLVr6HW1yLokcnhfURsn5zmSVdi3/xWWu1A==", + "requires": { + "ccount": "^1.0.0", + "mdast-util-find-and-replace": "^1.1.0", + "micromark": "^2.11.3" + } + }, + "mdast-util-gfm-strikethrough": { + "version": "0.2.3", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-strikethrough/-/mdast-util-gfm-strikethrough-0.2.3.tgz", + "integrity": "sha512-5OQLXpt6qdbttcDG/UxYY7Yjj3e8P7X16LzvpX8pIQPYJ/C2Z1qFGMmcw+1PZMUM3Z8wt8NRfYTvCni93mgsgA==", + "requires": { + "mdast-util-to-markdown": "^0.6.0" + } + }, + "mdast-util-gfm-table": { + "version": "0.1.6", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-table/-/mdast-util-gfm-table-0.1.6.tgz", + "integrity": "sha512-j4yDxQ66AJSBwGkbpFEp9uG/LS1tZV3P33fN1gkyRB2LoRL+RR3f76m0HPHaby6F4Z5xr9Fv1URmATlRRUIpRQ==", + "requires": { + "markdown-table": "^2.0.0", + "mdast-util-to-markdown": "~0.6.0" + } + }, + "mdast-util-gfm-task-list-item": { + "version": "0.1.6", + "resolved": "https://registry.npmmirror.com/mdast-util-gfm-task-list-item/-/mdast-util-gfm-task-list-item-0.1.6.tgz", + "integrity": "sha512-/d51FFIfPsSmCIRNp7E6pozM9z1GYPIkSy1urQ8s/o4TC22BZ7DqfHFWiqBD23bc7J3vV1Fc9O4QIHBlfuit8A==", + "requires": { + "mdast-util-to-markdown": "~0.6.0" + } + }, + "mdast-util-to-hast": { + "version": "10.2.0", + "resolved": "https://registry.npmmirror.com/mdast-util-to-hast/-/mdast-util-to-hast-10.2.0.tgz", + "integrity": "sha512-JoPBfJ3gBnHZ18icCwHR50orC9kNH81tiR1gs01D8Q5YpV6adHNO9nKNuFBCJQ941/32PT1a63UF/DitmS3amQ==", + "requires": { + "@types/mdast": "^3.0.0", + "@types/unist": "^2.0.0", + "mdast-util-definitions": "^4.0.0", + "mdurl": "^1.0.0", + "unist-builder": "^2.0.0", + "unist-util-generated": "^1.0.0", + "unist-util-position": "^3.0.0", + "unist-util-visit": "^2.0.0" + } + }, + "mdast-util-to-markdown": { + "version": "0.6.5", + "resolved": "https://registry.npmmirror.com/mdast-util-to-markdown/-/mdast-util-to-markdown-0.6.5.tgz", + "integrity": "sha512-XeV9sDE7ZlOQvs45C9UKMtfTcctcaj/pGwH8YLbMHoMOXNNCn2LsqVQOqrF1+/NU8lKDAqozme9SCXWyo9oAcQ==", + "requires": { + "@types/unist": "^2.0.0", + "longest-streak": "^2.0.0", + "mdast-util-to-string": "^2.0.0", + "parse-entities": "^2.0.0", + "repeat-string": "^1.0.0", + "zwitch": "^1.0.0" + } + }, + "mdast-util-to-nlcst": { + "version": "4.0.1", + "resolved": "https://registry.npmmirror.com/mdast-util-to-nlcst/-/mdast-util-to-nlcst-4.0.1.tgz", + "integrity": "sha512-Y4ffygj85MTt70STKnEquw6k73jYWJBaYcb4ITAKgSNokZF7fH8rEHZ1GsRY/JaxqUevMaEnsDmkVv5Z9uVRdg==", + "requires": { + "nlcst-to-string": "^2.0.0", + "repeat-string": "^1.0.0", + "unist-util-position": "^3.0.0", + "vfile-location": "^3.1.0" + } + }, + "mdast-util-to-string": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/mdast-util-to-string/-/mdast-util-to-string-2.0.0.tgz", + "integrity": "sha512-AW4DRS3QbBayY/jJmD8437V1Gombjf8RSOUCMFBuo5iHi58AGEgVCKQ+ezHkZZDpAQS75hcBMpLqjpJTjtUL7w==" + }, + "mdast-util-toc": { + "version": "5.1.0", + "resolved": "https://registry.npmmirror.com/mdast-util-toc/-/mdast-util-toc-5.1.0.tgz", + "integrity": "sha512-csimbRIVkiqc+PpFeKDGQ/Ck2N4f9FYH3zzBMMJzcxoKL8m+cM0n94xXm0I9eaxHnKdY9n145SGTdyJC7i273g==", + "requires": { + "@types/mdast": "^3.0.3", + "@types/unist": "^2.0.3", + "extend": "^3.0.2", + "github-slugger": "^1.2.1", + "mdast-util-to-string": "^2.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit": "^2.0.0" + } + }, + "mdn-data": { + "version": "2.0.14", + "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-2.0.14.tgz", + "integrity": "sha512-dn6wd0uw5GsdswPFfsgMp5NSB0/aDe6fK94YJV/AJDYXL6HVLWBsxeq7js7Ad+mU2K9LAlwpk6kN2D5mwCPVow==" + }, + "mdurl": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/mdurl/-/mdurl-1.0.1.tgz", + "integrity": "sha512-/sKlQJCBYVY9Ers9hqzKou4H6V5UWc/M59TH2dvkt+84itfnq7uFOMLpOiOS4ujvHP4etln18fmIxA5R5fll0g==" + }, + "meant": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/meant/-/meant-1.0.3.tgz", + "integrity": "sha512-88ZRGcNxAq4EH38cQ4D85PM57pikCwS8Z99EWHODxN7KBY+UuPiqzRTtZzS8KTXO/ywSWbdjjJST2Hly/EQxLw==" + }, + "media-typer": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/media-typer/-/media-typer-0.3.0.tgz", + "integrity": "sha1-hxDXrwqmJvj/+hzgAWhUUmMlV0g=" + }, + "mem": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/mem/-/mem-8.1.1.tgz", + "integrity": "sha512-qFCFUDs7U3b8mBDPyz5EToEKoAkgCzqquIgi9nkkR9bixxOVOre+09lbuH7+9Kn2NFpm56M3GUWVbU2hQgdACA==", + "requires": { + "map-age-cleaner": "^0.1.3", + "mimic-fn": "^3.1.0" + }, + "dependencies": { + "mimic-fn": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-3.1.0.tgz", + "integrity": "sha512-Ysbi9uYW9hFyfrThdDEQuykN4Ey6BuwPD2kpI5ES/nFTDn/98yxYNLZJcgUAKPT/mcrLLKaGzJR9YVxJrIdASQ==" + } + } + }, + "memfs": { + "version": "3.4.1", + "resolved": "https://registry.npmjs.org/memfs/-/memfs-3.4.1.tgz", + "integrity": "sha512-1c9VPVvW5P7I85c35zAdEr1TD5+F11IToIHIlrVIcflfnzPkJa0ZoYEoEdYDP8KgPFoSZ/opDrUsAoZWym3mtw==", + "requires": { + "fs-monkey": "1.0.3" + } + }, + "memoizee": { + "version": "0.4.15", + "resolved": "https://registry.npmjs.org/memoizee/-/memoizee-0.4.15.tgz", + "integrity": "sha512-UBWmJpLZd5STPm7PMUlOw/TSy972M+z8gcyQ5veOnSDRREz/0bmpyTfKt3/51DhEBqCZQn1udM/5flcSPYhkdQ==", + "requires": { + "d": "^1.0.1", + "es5-ext": "^0.10.53", + "es6-weak-map": "^2.0.3", + "event-emitter": "^0.3.5", + "is-promise": "^2.2.2", + "lru-queue": "^0.1.0", + "next-tick": "^1.1.0", + "timers-ext": "^0.1.7" + }, + "dependencies": { + "is-promise": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/is-promise/-/is-promise-2.2.2.tgz", + "integrity": "sha512-+lP4/6lKUBfQjZ2pdxThZvLUAafmZb8OAxFb8XXtiQmS35INgr85hdOGoEs124ez1FCnZJt6jau/T+alh58QFQ==" + } + } + }, + "meow": { + "version": "9.0.0", + "resolved": "https://registry.npmmirror.com/meow/-/meow-9.0.0.tgz", + "integrity": "sha512-+obSblOQmRhcyBt62furQqRAQpNyWXo8BuQ5bN7dG8wmwQ+vwHKp/rCFD4CrTP8CsDQD1sjoZ94K417XEUk8IQ==", + "requires": { + "@types/minimist": "^1.2.0", + "camelcase-keys": "^6.2.2", + "decamelize": "^1.2.0", + "decamelize-keys": "^1.1.0", + "hard-rejection": "^2.1.0", + "minimist-options": "4.1.0", + "normalize-package-data": "^3.0.0", + "read-pkg-up": "^7.0.1", + "redent": "^3.0.0", + "trim-newlines": "^3.0.0", + "type-fest": "^0.18.0", + "yargs-parser": "^20.2.3" + }, + "dependencies": { + "type-fest": { + "version": "0.18.1", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.18.1.tgz", + "integrity": "sha512-OIAYXk8+ISY+qTOwkHtKqzAuxchoMiD9Udx+FSGQDuiRR+PJKJHc2NJAXlbhkGwTt/4/nKZxELY1w3ReWOL8mw==" + }, + "yargs-parser": { + "version": "20.2.9", + "resolved": "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-20.2.9.tgz", + "integrity": "sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w==" + } + } + }, + "merge-descriptors": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/merge-descriptors/-/merge-descriptors-1.0.1.tgz", + "integrity": "sha1-sAqqVW3YtEVoFQ7J0blT8/kMu2E=" + }, + "merge-stream": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/merge-stream/-/merge-stream-2.0.0.tgz", + "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==" + }, + "merge2": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/merge2/-/merge2-1.4.1.tgz", + "integrity": "sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg==" + }, + "meros": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/meros/-/meros-1.1.4.tgz", + "integrity": "sha512-E9ZXfK9iQfG9s73ars9qvvvbSIkJZF5yOo9j4tcwM5tN8mUKfj/EKN5PzOr3ZH0y5wL7dLAHw3RVEfpQV9Q7VQ==", + "requires": {} + }, + "methods": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/methods/-/methods-1.1.2.tgz", + "integrity": "sha1-VSmk1nZUE07cxSZmVoNbD4Ua/O4=" + }, + "micromark": { + "version": "2.11.4", + "resolved": "https://registry.npmmirror.com/micromark/-/micromark-2.11.4.tgz", + "integrity": "sha512-+WoovN/ppKolQOFIAajxi7Lu9kInbPxFuTBVEavFcL8eAfVstoc5MocPmqBeAdBOJV00uaVjegzH4+MA0DN/uA==", + "requires": { + "debug": "^4.0.0", + "parse-entities": "^2.0.0" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmmirror.com/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "micromark-extension-footnote": { + "version": "0.3.2", + "resolved": "https://registry.npmmirror.com/micromark-extension-footnote/-/micromark-extension-footnote-0.3.2.tgz", + "integrity": "sha512-gr/BeIxbIWQoUm02cIfK7mdMZ/fbroRpLsck4kvFtjbzP4yi+OPVbnukTc/zy0i7spC2xYE/dbX1Sur8BEDJsQ==", + "requires": { + "micromark": "~2.11.0" + } + }, + "micromark-extension-gfm": { + "version": "0.3.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm/-/micromark-extension-gfm-0.3.3.tgz", + "integrity": "sha512-oVN4zv5/tAIA+l3GbMi7lWeYpJ14oQyJ3uEim20ktYFAcfX1x3LNlFGGlmrZHt7u9YlKExmyJdDGaTt6cMSR/A==", + "requires": { + "micromark": "~2.11.0", + "micromark-extension-gfm-autolink-literal": "~0.5.0", + "micromark-extension-gfm-strikethrough": "~0.6.5", + "micromark-extension-gfm-table": "~0.4.0", + "micromark-extension-gfm-tagfilter": "~0.3.0", + "micromark-extension-gfm-task-list-item": "~0.3.0" + } + }, + "micromark-extension-gfm-autolink-literal": { + "version": "0.5.7", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-autolink-literal/-/micromark-extension-gfm-autolink-literal-0.5.7.tgz", + "integrity": "sha512-ePiDGH0/lhcngCe8FtH4ARFoxKTUelMp4L7Gg2pujYD5CSMb9PbblnyL+AAMud/SNMyusbS2XDSiPIRcQoNFAw==", + "requires": { + "micromark": "~2.11.3" + } + }, + "micromark-extension-gfm-strikethrough": { + "version": "0.6.5", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-strikethrough/-/micromark-extension-gfm-strikethrough-0.6.5.tgz", + "integrity": "sha512-PpOKlgokpQRwUesRwWEp+fHjGGkZEejj83k9gU5iXCbDG+XBA92BqnRKYJdfqfkrRcZRgGuPuXb7DaK/DmxOhw==", + "requires": { + "micromark": "~2.11.0" + } + }, + "micromark-extension-gfm-table": { + "version": "0.4.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-table/-/micromark-extension-gfm-table-0.4.3.tgz", + "integrity": "sha512-hVGvESPq0fk6ALWtomcwmgLvH8ZSVpcPjzi0AjPclB9FsVRgMtGZkUcpE0zgjOCFAznKepF4z3hX8z6e3HODdA==", + "requires": { + "micromark": "~2.11.0" + } + }, + "micromark-extension-gfm-tagfilter": { + "version": "0.3.0", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-tagfilter/-/micromark-extension-gfm-tagfilter-0.3.0.tgz", + "integrity": "sha512-9GU0xBatryXifL//FJH+tAZ6i240xQuFrSL7mYi8f4oZSbc+NvXjkrHemeYP0+L4ZUT+Ptz3b95zhUZnMtoi/Q==" + }, + "micromark-extension-gfm-task-list-item": { + "version": "0.3.3", + "resolved": "https://registry.npmmirror.com/micromark-extension-gfm-task-list-item/-/micromark-extension-gfm-task-list-item-0.3.3.tgz", + "integrity": "sha512-0zvM5iSLKrc/NQl84pZSjGo66aTGd57C1idmlWmE87lkMcXrTxg1uXa/nXomxJytoje9trP0NDLvw4bZ/Z/XCQ==", + "requires": { + "micromark": "~2.11.0" + } + }, + "micromatch": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.5.tgz", + "integrity": "sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA==", + "requires": { + "braces": "^3.0.2", + "picomatch": "^2.3.1" + } + }, + "mime": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-2.6.0.tgz", + "integrity": "sha512-USPkMeET31rOMiarsBNIHZKLGgvKc/LrjofAnBlOttf5ajRvqiRA8QsenbcooctK6d6Ts6aqZXBA+XbkKthiQg==" + }, + "mime-db": { + "version": "1.52.0", + "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz", + "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==" + }, + "mime-types": { + "version": "2.1.35", + "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz", + "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==", + "requires": { + "mime-db": "1.52.0" + } + }, + "mimic-fn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-2.1.0.tgz", + "integrity": "sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==" + }, + "mimic-response": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/mimic-response/-/mimic-response-1.0.1.tgz", + "integrity": "sha512-j5EctnkH7amfV/q5Hgmoal1g2QHFJRraOtmx0JpIqkxhBhI/lJSl1nMpQ45hVarwNETOoWEimndZ4QK0RHxuxQ==" + }, + "min-document": { + "version": "2.19.0", + "resolved": "https://registry.npmjs.org/min-document/-/min-document-2.19.0.tgz", + "integrity": "sha1-e9KC4/WELtKVu3SM3Z8f+iyCRoU=", + "requires": { + "dom-walk": "^0.1.0" + } + }, + "min-indent": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/min-indent/-/min-indent-1.0.1.tgz", + "integrity": "sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg==" + }, + "mini-css-extract-plugin": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/mini-css-extract-plugin/-/mini-css-extract-plugin-1.6.2.tgz", + "integrity": "sha512-WhDvO3SjGm40oV5y26GjMJYjd2UMqrLAGKy5YS2/3QKJy2F7jgynuHTir/tgUUOiNQu5saXHdc8reo7YuhhT4Q==", + "requires": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0", + "webpack-sources": "^1.1.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "mini-svg-data-uri": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/mini-svg-data-uri/-/mini-svg-data-uri-1.4.4.tgz", + "integrity": "sha512-r9deDe9p5FJUPZAk3A59wGH7Ii9YrjjWw0jmw/liSbHl2CHiyXj6FcDXDu2K3TjVAXqiJdaw3xxwlZZr9E6nHg==" + }, + "minimatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", + "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "requires": { + "brace-expansion": "^1.1.7" + } + }, + "minimist": { + "version": "1.2.6", + "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.6.tgz", + "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==" + }, + "minimist-options": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/minimist-options/-/minimist-options-4.1.0.tgz", + "integrity": "sha512-Q4r8ghd80yhO/0j1O3B2BjweX3fiHg9cdOwjJd2J76Q135c+NDxGCqdYKQ1SKBuFfgWbAUzBfvYjPUEeNgqN1A==", + "requires": { + "arrify": "^1.0.1", + "is-plain-obj": "^1.1.0", + "kind-of": "^6.0.3" + }, + "dependencies": { + "arrify": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/arrify/-/arrify-1.0.1.tgz", + "integrity": "sha512-3CYzex9M9FGQjCGMGyi6/31c8GJbgb0qGyrx5HWxPd0aCwh4cB2YjMb2Xf9UuoogrMrlO9cTqnB5rI5GHZTcUA==" + } + } + }, + "minipass": { + "version": "3.1.6", + "resolved": "https://registry.npmmirror.com/minipass/-/minipass-3.1.6.tgz", + "integrity": "sha512-rty5kpw9/z8SX9dmxblFA6edItUmwJgMeYDZRrwlIVN27i8gysGbznJwUggw2V/FVqFSDdWy040ZPS811DYAqQ==", + "requires": { + "yallist": "^4.0.0" + } + }, + "minizlib": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/minizlib/-/minizlib-2.1.2.tgz", + "integrity": "sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg==", + "requires": { + "minipass": "^3.0.0", + "yallist": "^4.0.0" + } + }, + "mitt": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/mitt/-/mitt-1.2.0.tgz", + "integrity": "sha512-r6lj77KlwqLhIUku9UWYes7KJtsczvolZkzp8hbaDPPaE24OmWl5s539Mytlj22siEQKosZ26qCBgda2PKwoJw==" + }, + "mkdirp": { + "version": "0.5.6", + "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-0.5.6.tgz", + "integrity": "sha512-FP+p8RB8OWpF3YZBCrP5gtADmtXApB5AMLn+vdyA+PyxCjrCs00mjyUozssO33cwDeT3wNGdLxJ5M//YqtHAJw==", + "requires": { + "minimist": "^1.2.6" + } + }, + "mkdirp-classic": { + "version": "0.5.3", + "resolved": "https://registry.npmjs.org/mkdirp-classic/-/mkdirp-classic-0.5.3.tgz", + "integrity": "sha512-gKLcREMhtuZRwRAfqP3RFW+TK4JqApVBtOIftVgjuABpAtpxhPGaDcfvbhNvD0B8iD1oUr/txX35NjcaY6Ns/A==" + }, + "moment": { + "version": "2.29.3", + "resolved": "https://registry.npmjs.org/moment/-/moment-2.29.3.tgz", + "integrity": "sha512-c6YRvhEo//6T2Jz/vVtYzqBzwvPT95JBQ+smCytzf7c50oMZRsR/a4w88aD34I+/QVSfnoAnSBFPJHItlOMJVw==" + }, + "ms": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", + "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" + }, + "msgpackr": { + "version": "1.5.7", + "resolved": "https://registry.npmjs.org/msgpackr/-/msgpackr-1.5.7.tgz", + "integrity": "sha512-Hsa80i8W4BiObSMHslfnwC+CC1CYHZzoXJZn0+3EvoCEOgt3c5QlXhdcjgFk2aZxMgpV8aUFZqJyQUCIp4UrzA==", + "requires": { + "msgpackr-extract": "^1.1.4" + } + }, + "msgpackr-extract": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/msgpackr-extract/-/msgpackr-extract-1.1.4.tgz", + "integrity": "sha512-WQbHvsThprXh+EqZYy+SQFEs7z6bNM7a0vgirwUfwUcphWGT2mdPcpyLCNiRsN6w5q5VKJUMblHY+tNEyceb9Q==", + "optional": true, + "requires": { + "msgpackr-extract-darwin-arm64": "1.1.0", + "msgpackr-extract-darwin-x64": "1.1.0", + "msgpackr-extract-linux-arm": "1.1.0", + "msgpackr-extract-linux-arm64": "1.1.0", + "msgpackr-extract-linux-x64": "1.1.0", + "msgpackr-extract-win32-x64": "1.1.0", + "node-gyp-build-optional-packages": "^4.3.2" + }, + "dependencies": { + "node-gyp-build-optional-packages": { + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/node-gyp-build-optional-packages/-/node-gyp-build-optional-packages-4.3.2.tgz", + "integrity": "sha512-P5Ep3ISdmwcCkZIaBaQamQtWAG0facC89phWZgi5Z3hBU//J6S48OIvyZWSPPf6yQMklLZiqoosWAZUj7N+esA==", + "optional": true + } + } + }, + "msgpackr-extract-darwin-arm64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-darwin-arm64/-/msgpackr-extract-darwin-arm64-1.1.0.tgz", + "integrity": "sha512-s1kHoT12tS2cCQOv+Wl3I+/cYNJXBPtwQqGA+dPYoXmchhXiE0Nso+BIfvQ5PxbmAyjj54Q5o7PnLTqVquNfZA==", + "optional": true + }, + "msgpackr-extract-darwin-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-darwin-x64/-/msgpackr-extract-darwin-x64-1.1.0.tgz", + "integrity": "sha512-yx/H/i12IKg4eWGu/eKdKzJD4jaYvvujQSaVmeOMCesbSQnWo5X6YR9TFjoiNoU9Aexk1KufzL9gW+1DozG1yw==", + "optional": true + }, + "msgpackr-extract-linux-arm": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-arm/-/msgpackr-extract-linux-arm-1.1.0.tgz", + "integrity": "sha512-0VvSCqi12xpavxl14gMrauwIzHqHbmSChUijy/uo3mpjB1Pk4vlisKpZsaOZvNJyNKj0ACi5jYtbWnnOd7hYGw==", + "optional": true + }, + "msgpackr-extract-linux-arm64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-arm64/-/msgpackr-extract-linux-arm64-1.1.0.tgz", + "integrity": "sha512-AxFle3fHNwz2V4CYDIGFxI6o/ZuI0lBKg0uHI8EcCMUmDE5mVAUWYge5WXmORVvb8sVWyVgFlmi3MTu4Ve6tNQ==", + "optional": true + }, + "msgpackr-extract-linux-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-linux-x64/-/msgpackr-extract-linux-x64-1.1.0.tgz", + "integrity": "sha512-O+XoyNFWpdB8oQL6O/YyzffPpmG5rTNrr1nKLW70HD2ENJUhcITzbV7eZimHPzkn8LAGls1tBaMTHQezTBpFOw==", + "optional": true + }, + "msgpackr-extract-win32-x64": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/msgpackr-extract-win32-x64/-/msgpackr-extract-win32-x64-1.1.0.tgz", + "integrity": "sha512-6AJdM5rNsL4yrskRfhujVSPEd6IBpgvsnIT/TPowKNLQ62iIdryizPY2PJNFiW3AJcY249AHEiDBXS1cTDPxzA==", + "optional": true + }, + "multer": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/multer/-/multer-1.4.4.tgz", + "integrity": "sha512-2wY2+xD4udX612aMqMcB8Ws2Voq6NIUPEtD1be6m411T4uDH/VtL9i//xvcyFlTVfRdaBsk7hV5tgrGQqhuBiw==", + "requires": { + "append-field": "^1.0.0", + "busboy": "^0.2.11", + "concat-stream": "^1.5.2", + "mkdirp": "^0.5.4", + "object-assign": "^4.1.1", + "on-finished": "^2.3.0", + "type-is": "^1.6.4", + "xtend": "^4.0.0" + } + }, + "multipipe": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/multipipe/-/multipipe-1.0.2.tgz", + "integrity": "sha512-6uiC9OvY71vzSGX8lZvSqscE7ft9nPupJ8fMjrCNRAUy2LREUW42UL+V/NTrogr6rFgRydUrCX4ZitfpSNkSCQ==", + "requires": { + "duplexer2": "^0.1.2", + "object-assign": "^4.1.0" + } + }, + "mute-stream": { + "version": "0.0.8", + "resolved": "https://registry.npmjs.org/mute-stream/-/mute-stream-0.0.8.tgz", + "integrity": "sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA==" + }, + "nan": { + "version": "2.15.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.15.0.tgz", + "integrity": "sha512-8ZtvEnA2c5aYCZYd1cvgdnU6cqwixRoYg70xPLWUws5ORTa/lnw+u4amixRS/Ac5U5mQVgp9pnlSUnbNWFaWZQ==" + }, + "nanoid": { + "version": "3.3.4", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.4.tgz", + "integrity": "sha512-MqBkQh/OHTS2egovRtLk45wEyNXwF+cokD+1YPf9u5VfJiRdAiRwB2froX5Co9Rh20xs4siNPm8naNotSD6RBw==" + }, + "napi-build-utils": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/napi-build-utils/-/napi-build-utils-1.0.2.tgz", + "integrity": "sha512-ONmRUqK7zj7DWX0D9ADe03wbwOBZxNAfF20PlGfCWQcD3+/MakShIHrMqx9YwPTfxDdF1zLeL+RGZiR9kGMLdg==" + }, + "native-url": { + "version": "0.2.6", + "resolved": "https://registry.npmjs.org/native-url/-/native-url-0.2.6.tgz", + "integrity": "sha512-k4bDC87WtgrdD362gZz6zoiXQrl40kYlBmpfmSjwRO1VU0V5ccwJTlxuE72F6m3V0vc1xOf6n3UCP9QyerRqmA==", + "requires": { + "querystring": "^0.2.0" + } + }, + "natural-compare": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", + "integrity": "sha1-Sr6/7tdUHywnrPspvbvRXI1bpPc=" + }, + "needle": { + "version": "2.9.1", + "resolved": "https://registry.npmjs.org/needle/-/needle-2.9.1.tgz", + "integrity": "sha512-6R9fqJ5Zcmf+uYaFgdIHmLwNldn5HbK8L5ybn7Uz+ylX/rnOsSp1AHcvQSrCaFN+qNM1wpymHqD7mVasEOlHGQ==", + "requires": { + "debug": "^3.2.6", + "iconv-lite": "^0.4.4", + "sax": "^1.2.4" + } + }, + "negotiator": { + "version": "0.6.3", + "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.6.3.tgz", + "integrity": "sha512-+EUsqGPLsM+j/zdChZjsnX51g4XrHFOIXwfnCVPGlQk/k5giakcKsuxCObBRu6DSm9opw/O6slWbJdghQM4bBg==" + }, + "neo-async": { + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/neo-async/-/neo-async-2.6.2.tgz", + "integrity": "sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw==" + }, + "next-tick": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/next-tick/-/next-tick-1.1.0.tgz", + "integrity": "sha512-CXdUiJembsNjuToQvxayPZF9Vqht7hewsvy2sOWafLvi2awflj9mOC6bHIg50orX8IJvWKY9wYQ/zB2kogPslQ==" + }, + "nice-try": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/nice-try/-/nice-try-1.0.5.tgz", + "integrity": "sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ==" + }, + "nlcst-to-string": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/nlcst-to-string/-/nlcst-to-string-2.0.4.tgz", + "integrity": "sha512-3x3jwTd6UPG7vi5k4GEzvxJ5rDA7hVUIRNHPblKuMVP9Z3xmlsd9cgLcpAMkc5uPOBna82EeshROFhsPkbnTZg==" + }, + "no-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/no-case/-/no-case-3.0.4.tgz", + "integrity": "sha512-fgAN3jGAh+RoxUGZHTSOLJIqUc2wmoBwGR4tbpNAKmmovFoWq0OdRkb0VkldReO2a2iBT/OEulG9XSUc10r3zg==", + "requires": { + "lower-case": "^2.0.2", + "tslib": "^2.0.3" + } + }, + "node-abi": { + "version": "3.15.0", + "resolved": "https://registry.npmjs.org/node-abi/-/node-abi-3.15.0.tgz", + "integrity": "sha512-Ic6z/j6I9RLm4ov7npo1I48UQr2BEyFCqh6p7S1dhEx9jPO0GPGq/e2Rb7x7DroQrmiVMz/Bw1vJm9sPAl2nxA==", + "requires": { + "semver": "^7.3.5" + } + }, + "node-addon-api": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/node-addon-api/-/node-addon-api-3.2.1.tgz", + "integrity": "sha512-mmcei9JghVNDYydghQmeDX8KoAm0FAiYyIcUt/N4nhyAipB17pllZQDOJD2fotxABnt4Mdz+dKTO7eftLg4d0A==" + }, + "node-fetch": { + "version": "2.6.7", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", + "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", + "requires": { + "whatwg-url": "^5.0.0" + } + }, + "node-gyp": { + "version": "7.1.2", + "resolved": "https://registry.npmmirror.com/node-gyp/-/node-gyp-7.1.2.tgz", + "integrity": "sha512-CbpcIo7C3eMu3dL1c3d0xw449fHIGALIJsRP4DDPHpyiW8vcriNY7ubh9TE4zEKfSxscY7PjeFnshE7h75ynjQ==", + "requires": { + "env-paths": "^2.2.0", + "glob": "^7.1.4", + "graceful-fs": "^4.2.3", + "nopt": "^5.0.0", + "npmlog": "^4.1.2", + "request": "^2.88.2", + "rimraf": "^3.0.2", + "semver": "^7.3.2", + "tar": "^6.0.2", + "which": "^2.0.2" + }, + "dependencies": { + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmmirror.com/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "requires": { + "isexe": "^2.0.0" + } + } + } + }, + "node-gyp-build": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/node-gyp-build/-/node-gyp-build-4.4.0.tgz", + "integrity": "sha512-amJnQCcgtRVw9SvoebO3BKGESClrfXGCUTX9hSn1OuGQTQBOZmVd0Z0OlecpuRksKvbsUqALE8jls/ErClAPuQ==" + }, + "node-int64": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/node-int64/-/node-int64-0.4.0.tgz", + "integrity": "sha1-h6kGXNs1XTGC2PlM4RGIuCXGijs=" + }, + "node-object-hash": { + "version": "2.3.10", + "resolved": "https://registry.npmjs.org/node-object-hash/-/node-object-hash-2.3.10.tgz", + "integrity": "sha512-jY5dPJzw6NHd/KPSfPKJ+IHoFS81/tJ43r34ZeNMXGzCOM8jwQDCD12HYayKIB6MuznrnqIYy2e891NA2g0ibA==" + }, + "node-releases": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.4.tgz", + "integrity": "sha512-gbMzqQtTtDz/00jQzZ21PQzdI9PyLYqUSvD0p3naOhX4odFji0ZxYdnVwPTxmSwkmxhcFImpozceidSG+AgoPQ==" + }, + "node-sass": { + "version": "6.0.1", + "resolved": "https://registry.npmmirror.com/node-sass/-/node-sass-6.0.1.tgz", + "integrity": "sha512-f+Rbqt92Ful9gX0cGtdYwjTrWAaGURgaK5rZCWOgCNyGWusFYHhbqCCBoFBeat+HKETOU02AyTxNhJV0YZf2jQ==", + "requires": { + "async-foreach": "^0.1.3", + "chalk": "^1.1.1", + "cross-spawn": "^7.0.3", + "gaze": "^1.0.0", + "get-stdin": "^4.0.1", + "glob": "^7.0.3", + "lodash": "^4.17.15", + "meow": "^9.0.0", + "nan": "^2.13.2", + "node-gyp": "^7.1.0", + "npmlog": "^4.0.0", + "request": "^2.88.0", + "sass-graph": "2.2.5", + "stdout-stream": "^1.4.0", + "true-case-path": "^1.0.2" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha512-TIGnTpdo+E3+pCyAluZvtED5p5wCqLdezCyhPZzKPcxvFplEt4i+W7OONCKgeZFT3+y5NZZfOOS/Bdcanm1MYA==" + }, + "ansi-styles": { + "version": "2.2.1", + "resolved": "https://registry.npmmirror.com/ansi-styles/-/ansi-styles-2.2.1.tgz", + "integrity": "sha512-kmCevFghRiWM7HB5zTPULl4r9bVFSWjz62MhqizDGUrq2NWuNMQyuv4tHHoKJHs69M/MF64lEcHdYIocrdWQYA==" + }, + "chalk": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/chalk/-/chalk-1.1.3.tgz", + "integrity": "sha512-U3lRVLMSlsCfjqYPbLyVv11M9CPW4I728d6TCKMAOJueEeB9/8o+eSsMnxPJD+Q+K909sdESg7C+tIkoH6on1A==", + "requires": { + "ansi-styles": "^2.2.1", + "escape-string-regexp": "^1.0.2", + "has-ansi": "^2.0.0", + "strip-ansi": "^3.0.0", + "supports-color": "^2.0.0" + } + }, + "cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmmirror.com/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "requires": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + } + }, + "path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==" + }, + "shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "requires": { + "shebang-regex": "^3.0.0" + } + }, + "shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==" + }, + "strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha512-VhumSSbBqDTP8p2ZLKj40UjBCV4+v8bUSEpUb4KjRgWk9pbqGF4REFj6KEagidb2f/M6AzC0EmFyDNGaw9OCzg==", + "requires": { + "ansi-regex": "^2.0.0" + } + }, + "supports-color": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/supports-color/-/supports-color-2.0.0.tgz", + "integrity": "sha512-KKNVtd6pCYgPIKU4cp2733HWYCpplQhddZLBUryaAHou723x+FRzQ5Df824Fj+IyyuiQTRoub4SnIFfIcrp70g==" + }, + "true-case-path": { + "version": "1.0.3", + "resolved": "https://registry.npmmirror.com/true-case-path/-/true-case-path-1.0.3.tgz", + "integrity": "sha512-m6s2OdQe5wgpFMC+pAJ+q9djG82O2jcHPOI6RNg1yy9rCYR+WD6Nbpl32fDpfC56nirdRy+opFa/Vk7HYhqaew==", + "requires": { + "glob": "^7.1.2" + } + }, + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmmirror.com/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "requires": { + "isexe": "^2.0.0" + } + } + } + }, + "nopt": { + "version": "5.0.0", + "resolved": "https://registry.npmmirror.com/nopt/-/nopt-5.0.0.tgz", + "integrity": "sha512-Tbj67rffqceeLpcRXrT7vKAN8CwfPeIBgM7E6iBkmKLV7bEMwpGgYLGv0jACUsECaa/vuxP0IjEont6umdMgtQ==", + "requires": { + "abbrev": "1" + } + }, + "normalize-package-data": { + "version": "3.0.3", + "resolved": "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-3.0.3.tgz", + "integrity": "sha512-p2W1sgqij3zMMyRC067Dg16bfzVH+w7hyegmpIvZ4JNjqtGOVAIvLmjBx3yP7YTe9vKJgkoNOPjwQGogDoMXFA==", + "requires": { + "hosted-git-info": "^4.0.1", + "is-core-module": "^2.5.0", + "semver": "^7.3.4", + "validate-npm-package-license": "^3.0.1" + }, + "dependencies": { + "hosted-git-info": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-4.1.0.tgz", + "integrity": "sha512-kyCuEOWjJqZuDbRHzL8V93NzQhwIB71oFWSyzVo+KPZI+pnQPPxucdkrOZvkLRnrf5URsQM+IJ09Dw29cRALIA==", + "requires": { + "lru-cache": "^6.0.0" + } + } + } + }, + "normalize-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", + "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==" + }, + "normalize-range": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/normalize-range/-/normalize-range-0.1.2.tgz", + "integrity": "sha1-LRDAa9/TEuqXd2laTShDlFa3WUI=" + }, + "normalize-url": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-6.1.0.tgz", + "integrity": "sha512-DlL+XwOy3NxAQ8xuC0okPgK46iuVNAK01YN7RueYBqqFeGsBjV9XmCAzAdgt+667bCl5kPh9EqKKDwnaPG1I7A==" + }, + "not": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/not/-/not-0.1.0.tgz", + "integrity": "sha512-5PDmaAsVfnWUgTUbJ3ERwn7u79Z0dYxN9ErxCpVJJqe2RK0PJ3z+iFUxuqjwtlDDegXvtWoxD/3Fzxox7tFGWA==" + }, + "npm-run-path": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-4.0.1.tgz", + "integrity": "sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw==", + "requires": { + "path-key": "^3.0.0" + }, + "dependencies": { + "path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==" + } + } + }, + "npmlog": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-4.1.2.tgz", + "integrity": "sha512-2uUqazuKlTaSI/dC8AzicUck7+IrEaOnN/e0jd3Xtt1KcGpwx30v50mL7oPyr/h9bL3E4aZccVwpwP+5W9Vjkg==", + "requires": { + "are-we-there-yet": "~1.1.2", + "console-control-strings": "~1.1.0", + "gauge": "~2.7.3", + "set-blocking": "~2.0.0" + } + }, + "nth-check": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/nth-check/-/nth-check-2.0.1.tgz", + "integrity": "sha512-it1vE95zF6dTT9lBsYbxvqh0Soy4SPowchj0UBGj/V6cTPnXXtQOPUbhZ6CmGzAD/rW22LQK6E96pcdJXk4A4w==", + "requires": { + "boolbase": "^1.0.0" + } + }, + "null-loader": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/null-loader/-/null-loader-4.0.1.tgz", + "integrity": "sha512-pxqVbi4U6N26lq+LmgIbB5XATP0VdZKOG25DhHi8btMmJJefGArFyDg1yc4U3hWCJbMqSrw0qyrz1UQX+qYXqg==", + "requires": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "nullthrows": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/nullthrows/-/nullthrows-1.1.1.tgz", + "integrity": "sha512-2vPPEi+Z7WqML2jZYddDIfy5Dqb0r2fze2zTxNNknZaFpVHU3mFB3R+DWeJWGVx0ecvttSGlJTI+WG+8Z4cDWw==" + }, + "number-is-nan": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/number-is-nan/-/number-is-nan-1.0.1.tgz", + "integrity": "sha1-CXtgK1NCKlIsGvuHkDGDNpQaAR0=" + }, + "oauth-sign": { + "version": "0.9.0", + "resolved": "https://registry.npmmirror.com/oauth-sign/-/oauth-sign-0.9.0.tgz", + "integrity": "sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ==" + }, + "object-assign": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", + "integrity": "sha1-IQmtx5ZYh8/AXLvUQsrIv7s2CGM=" + }, + "object-inspect": { + "version": "1.12.0", + "resolved": "https://registry.npmjs.org/object-inspect/-/object-inspect-1.12.0.tgz", + "integrity": "sha512-Ho2z80bVIvJloH+YzRmpZVQe87+qASmBUKZDWgx9cu+KDrX2ZDH/3tMy+gXbZETVGs2M8YdxObOh7XAtim9Y0g==" + }, + "object-keys": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", + "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==" + }, + "object.assign": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/object.assign/-/object.assign-4.1.2.tgz", + "integrity": "sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ==", + "requires": { + "call-bind": "^1.0.0", + "define-properties": "^1.1.3", + "has-symbols": "^1.0.1", + "object-keys": "^1.1.1" + } + }, + "object.entries": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/object.entries/-/object.entries-1.1.5.tgz", + "integrity": "sha512-TyxmjUoZggd4OrrU1W66FMDG6CuqJxsFvymeyXI51+vQLN67zYfZseptRge703kKQdo4uccgAKebXFcRCzk4+g==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + } + }, + "object.fromentries": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/object.fromentries/-/object.fromentries-2.0.5.tgz", + "integrity": "sha512-CAyG5mWQRRiBU57Re4FKoTBjXfDoNwdFVH2Y1tS9PqCsfUTymAohOkEMSG3aRNKmv4lV3O7p1et7c187q6bynw==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + } + }, + "object.getownpropertydescriptors": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/object.getownpropertydescriptors/-/object.getownpropertydescriptors-2.1.3.tgz", + "integrity": "sha512-VdDoCwvJI4QdC6ndjpqFmoL3/+HxffFBbcJzKi5hwLLqqx3mdbedRpfZDdK0SrOSauj8X4GzBvnDZl4vTN7dOw==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + } + }, + "object.hasown": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/object.hasown/-/object.hasown-1.1.1.tgz", + "integrity": "sha512-LYLe4tivNQzq4JdaWW6WO3HMZZJWzkkH8fnI6EebWl0VZth2wL2Lovm74ep2/gZzlaTdV62JZHEqHQ2yVn8Q/A==", + "requires": { + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "object.values": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/object.values/-/object.values-1.1.5.tgz", + "integrity": "sha512-QUZRW0ilQ3PnPpbNtgdNV1PDbEqLIiSFB3l+EnGtBQ/8SUTLj1PZwtQHABZtLgwpJZTSZhuGLOGk57Drx2IvYg==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1" + } + }, + "objectFitPolyfill": { + "version": "2.3.5", + "resolved": "https://registry.npmjs.org/objectFitPolyfill/-/objectFitPolyfill-2.3.5.tgz", + "integrity": "sha512-8Quz071ZmGi0QWEG4xB3Bv5Lpw6K0Uca87FLoLMKMWjB6qIq9IyBegP3b/VLNxv2WYvIMGoeUQ+c6ibUkNa8TA==" + }, + "omggif": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/omggif/-/omggif-1.0.10.tgz", + "integrity": "sha512-LMJTtvgc/nugXj0Vcrrs68Mn2D1r0zf630VNtqtpI1FEO7e+O9FP4gqs9AcnBaSEeoHIPm28u6qgPR0oyEpGSw==" + }, + "on-finished": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/on-finished/-/on-finished-2.4.1.tgz", + "integrity": "sha512-oVlzkg3ENAhCk2zdv7IJwd/QUD4z2RxRwpkcGY8psCVcCYZNq4wYnVWALHM+brtuJjePWiYF/ClmuDr8Ch5+kg==", + "requires": { + "ee-first": "1.1.1" + } + }, + "on-headers": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/on-headers/-/on-headers-1.0.2.tgz", + "integrity": "sha512-pZAE+FJLoyITytdqK0U5s+FIpjN0JP3OzFi/u8Rx+EV5/W+JTWGXG8xFzevE7AjBfDqHv/8vL8qQsIhHnqRkrA==" + }, + "once": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", + "integrity": "sha1-WDsap3WWHUsROsF9nFC6753Xa9E=", + "requires": { + "wrappy": "1" + } + }, + "onetime": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/onetime/-/onetime-5.1.2.tgz", + "integrity": "sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg==", + "requires": { + "mimic-fn": "^2.1.0" + } + }, + "open": { + "version": "7.4.2", + "resolved": "https://registry.npmjs.org/open/-/open-7.4.2.tgz", + "integrity": "sha512-MVHddDVweXZF3awtlAS+6pgKLlm/JgxZ90+/NBurBoQctVOOB/zDdVjcyPzQ+0laDGbsWgrRkflI65sQeOgT9Q==", + "requires": { + "is-docker": "^2.0.0", + "is-wsl": "^2.1.1" + } + }, + "opentracing": { + "version": "0.14.7", + "resolved": "https://registry.npmjs.org/opentracing/-/opentracing-0.14.7.tgz", + "integrity": "sha512-vz9iS7MJ5+Bp1URw8Khvdyw1H/hGvzHWlKQ7eRrQojSCDL1/SrWfrY9QebLw97n2deyRtzHRC3MkQfVNUCo91Q==" + }, + "optionator": { + "version": "0.9.1", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.1.tgz", + "integrity": "sha512-74RlY5FCnhq4jRxVUPKDaRwrVNXMqsGsiW6AJw4XK8hmtm10wC0ypZBLw5IIp85NZMr91+qd1RvvENwg7jjRFw==", + "requires": { + "deep-is": "^0.1.3", + "fast-levenshtein": "^2.0.6", + "levn": "^0.4.1", + "prelude-ls": "^1.2.1", + "type-check": "^0.4.0", + "word-wrap": "^1.2.3" + } + }, + "ordered-binary": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/ordered-binary/-/ordered-binary-1.2.5.tgz", + "integrity": "sha512-djRmZoEpOGvIRW7ufsCDHtvcUa18UC9TxnPbHhSVFZHsoyg0dtut1bWtBZ/fmxdPN62oWXrV6adM7NoWU+CneA==" + }, + "os-tmpdir": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/os-tmpdir/-/os-tmpdir-1.0.2.tgz", + "integrity": "sha1-u+Z0BseaqFxc/sdm/lc0VV36EnQ=" + }, + "p-cancelable": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-2.1.1.tgz", + "integrity": "sha512-BZOr3nRQHOntUjTrH8+Lh54smKHoHyur8We1V8DSMVrl5A2malOOwuJRnKRDjSnkoeBh4at6BwEnb5I7Jl31wg==" + }, + "p-defer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/p-defer/-/p-defer-3.0.0.tgz", + "integrity": "sha512-ugZxsxmtTln604yeYd29EGrNhazN2lywetzpKhfmQjW/VJmhpDmWbiX+h0zL8V91R0UXkhb3KtPmyq9PZw3aYw==" + }, + "p-finally": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-finally/-/p-finally-1.0.0.tgz", + "integrity": "sha1-P7z7FbiZpEEjs0ttzBi3JDNqLK4=" + }, + "p-limit": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", + "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "requires": { + "p-try": "^2.0.0" + } + }, + "p-locate": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", + "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", + "requires": { + "p-limit": "^2.2.0" + } + }, + "p-try": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-2.2.0.tgz", + "integrity": "sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ==" + }, + "package-json": { + "version": "6.5.0", + "resolved": "https://registry.npmjs.org/package-json/-/package-json-6.5.0.tgz", + "integrity": "sha512-k3bdm2n25tkyxcjSKzB5x8kfVxlMdgsbPr0GkZcwHsLpba6cBjqCt1KlcChKEvxHIcTB1FVMuwoijZ26xex5MQ==", + "requires": { + "got": "^9.6.0", + "registry-auth-token": "^4.0.0", + "registry-url": "^5.0.0", + "semver": "^6.2.0" + }, + "dependencies": { + "@sindresorhus/is": { + "version": "0.14.0", + "resolved": "https://registry.npmjs.org/@sindresorhus/is/-/is-0.14.0.tgz", + "integrity": "sha512-9NET910DNaIPngYnLLPeg+Ogzqsi9uM4mSboU5y6p8S5DzMTVEsJZrawi+BoDNUVBa2DhJqQYUFvMDfgU062LQ==" + }, + "@szmarczak/http-timer": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@szmarczak/http-timer/-/http-timer-1.1.2.tgz", + "integrity": "sha512-XIB2XbzHTN6ieIjfIMV9hlVcfPU26s2vafYWQcZHWXHOxiaRZYEDKEwdl129Zyg50+foYV2jCgtrqSA6qNuNSA==", + "requires": { + "defer-to-connect": "^1.0.1" + } + }, + "cacheable-request": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/cacheable-request/-/cacheable-request-6.1.0.tgz", + "integrity": "sha512-Oj3cAGPCqOZX7Rz64Uny2GYAZNliQSqfbePrgAQ1wKAihYmCUnraBtJtKcGR4xz7wF+LoJC+ssFZvv5BgF9Igg==", + "requires": { + "clone-response": "^1.0.2", + "get-stream": "^5.1.0", + "http-cache-semantics": "^4.0.0", + "keyv": "^3.0.0", + "lowercase-keys": "^2.0.0", + "normalize-url": "^4.1.0", + "responselike": "^1.0.2" + }, + "dependencies": { + "get-stream": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz", + "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==", + "requires": { + "pump": "^3.0.0" + } + }, + "lowercase-keys": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-2.0.0.tgz", + "integrity": "sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA==" + } + } + }, + "decompress-response": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/decompress-response/-/decompress-response-3.3.0.tgz", + "integrity": "sha1-gKTdMjdIOEv6JICDYirt7Jgq3/M=", + "requires": { + "mimic-response": "^1.0.0" + } + }, + "defer-to-connect": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/defer-to-connect/-/defer-to-connect-1.1.3.tgz", + "integrity": "sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ==" + }, + "get-stream": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-4.1.0.tgz", + "integrity": "sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==", + "requires": { + "pump": "^3.0.0" + } + }, + "got": { + "version": "9.6.0", + "resolved": "https://registry.npmjs.org/got/-/got-9.6.0.tgz", + "integrity": "sha512-R7eWptXuGYxwijs0eV+v3o6+XH1IqVK8dJOEecQfTmkncw9AV4dcw/Dhxi8MdlqPthxxpZyizMzyg8RTmEsG+Q==", + "requires": { + "@sindresorhus/is": "^0.14.0", + "@szmarczak/http-timer": "^1.1.2", + "cacheable-request": "^6.0.0", + "decompress-response": "^3.3.0", + "duplexer3": "^0.1.4", + "get-stream": "^4.1.0", + "lowercase-keys": "^1.0.1", + "mimic-response": "^1.0.1", + "p-cancelable": "^1.0.0", + "to-readable-stream": "^1.0.0", + "url-parse-lax": "^3.0.0" + } + }, + "json-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.0.tgz", + "integrity": "sha1-Wx85evx11ne96Lz8Dkfh+aPZqJg=" + }, + "keyv": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-3.1.0.tgz", + "integrity": "sha512-9ykJ/46SN/9KPM/sichzQ7OvXyGDYKGTaDlKMGCAlg2UK8KRy4jb0d8sFc+0Tt0YYnThq8X2RZgCg74RPxgcVA==", + "requires": { + "json-buffer": "3.0.0" + } + }, + "lowercase-keys": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/lowercase-keys/-/lowercase-keys-1.0.1.tgz", + "integrity": "sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA==" + }, + "normalize-url": { + "version": "4.5.1", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-4.5.1.tgz", + "integrity": "sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA==" + }, + "p-cancelable": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-1.1.0.tgz", + "integrity": "sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw==" + }, + "responselike": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-1.0.2.tgz", + "integrity": "sha1-kYcg7ztjHFZCvgaPFa3lpG9Loec=", + "requires": { + "lowercase-keys": "^1.0.0" + } + }, + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "pako": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/pako/-/pako-1.0.11.tgz", + "integrity": "sha512-4hLB8Py4zZce5s4yd9XzopqwVv/yGNhV1Bl8NTmCq1763HeK2+EwVTv+leGeL13Dnh2wfbqowVPXCIO0z4taYw==" + }, + "param-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/param-case/-/param-case-3.0.4.tgz", + "integrity": "sha512-RXlj7zCYokReqWpOPH9oYivUzLYZ5vAPIfEmCTNViosC78F8F0H9y7T7gG2M39ymgutxF5gcFEsyZQSph9Bp3A==", + "requires": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "parent-module": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/parent-module/-/parent-module-1.0.1.tgz", + "integrity": "sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g==", + "requires": { + "callsites": "^3.0.0" + } + }, + "parse-bmfont-ascii": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/parse-bmfont-ascii/-/parse-bmfont-ascii-1.0.6.tgz", + "integrity": "sha1-Eaw8P/WPfCAgqyJ2kHkQjU36AoU=" + }, + "parse-bmfont-binary": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/parse-bmfont-binary/-/parse-bmfont-binary-1.0.6.tgz", + "integrity": "sha1-0Di0dtPp3Z2x4RoLDlOiJ5K2kAY=" + }, + "parse-bmfont-xml": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/parse-bmfont-xml/-/parse-bmfont-xml-1.1.4.tgz", + "integrity": "sha512-bjnliEOmGv3y1aMEfREMBJ9tfL3WR0i0CKPj61DnSLaoxWR3nLrsQrEbCId/8rF4NyRF0cCqisSVXyQYWM+mCQ==", + "requires": { + "xml-parse-from-string": "^1.0.0", + "xml2js": "^0.4.5" + } + }, + "parse-english": { + "version": "4.2.0", + "resolved": "https://registry.npmmirror.com/parse-english/-/parse-english-4.2.0.tgz", + "integrity": "sha512-jw5N6wZUZViIw3VLG/FUSeL3vDhfw5Q2g4E3nYC69Mm5ANbh9ZWd+eligQbeUoyObZM8neynTn3l14e09pjEWg==", + "requires": { + "nlcst-to-string": "^2.0.0", + "parse-latin": "^4.0.0", + "unist-util-modify-children": "^2.0.0", + "unist-util-visit-children": "^1.0.0" + } + }, + "parse-entities": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/parse-entities/-/parse-entities-2.0.0.tgz", + "integrity": "sha512-kkywGpCcRYhqQIchaWqZ875wzpS/bMKhz5HnN3p7wveJTkTtyAB/AlnS0f8DFSqYW1T82t6yEAkEcB+A1I3MbQ==", + "requires": { + "character-entities": "^1.0.0", + "character-entities-legacy": "^1.0.0", + "character-reference-invalid": "^1.0.0", + "is-alphanumerical": "^1.0.0", + "is-decimal": "^1.0.0", + "is-hexadecimal": "^1.0.0" + } + }, + "parse-filepath": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/parse-filepath/-/parse-filepath-1.0.2.tgz", + "integrity": "sha1-pjISf1Oq89FYdvWHLz/6x2PWyJE=", + "requires": { + "is-absolute": "^1.0.0", + "map-cache": "^0.2.0", + "path-root": "^0.1.1" + } + }, + "parse-headers": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/parse-headers/-/parse-headers-2.0.5.tgz", + "integrity": "sha512-ft3iAoLOB/MlwbNXgzy43SWGP6sQki2jQvAyBg/zDFAgr9bfNWZIUj42Kw2eJIl8kEi4PbgE6U1Zau/HwI75HA==" + }, + "parse-json": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/parse-json/-/parse-json-5.2.0.tgz", + "integrity": "sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg==", + "requires": { + "@babel/code-frame": "^7.0.0", + "error-ex": "^1.3.1", + "json-parse-even-better-errors": "^2.3.0", + "lines-and-columns": "^1.1.6" + } + }, + "parse-latin": { + "version": "4.3.0", + "resolved": "https://registry.npmmirror.com/parse-latin/-/parse-latin-4.3.0.tgz", + "integrity": "sha512-TYKL+K98dcAWoCw/Ac1yrPviU8Trk+/gmjQVaoWEFDZmVD4KRg6c/80xKqNNFQObo2mTONgF8trzAf2UTwKafw==", + "requires": { + "nlcst-to-string": "^2.0.0", + "unist-util-modify-children": "^2.0.0", + "unist-util-visit-children": "^1.0.0" + } + }, + "parse-path": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/parse-path/-/parse-path-4.0.3.tgz", + "integrity": "sha512-9Cepbp2asKnWTJ9x2kpw6Fe8y9JDbqwahGCTvklzd/cEq5C5JC59x2Xb0Kx+x0QZ8bvNquGO8/BWP0cwBHzSAA==", + "requires": { + "is-ssh": "^1.3.0", + "protocols": "^1.4.0", + "qs": "^6.9.4", + "query-string": "^6.13.8" + } + }, + "parse-srcset": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/parse-srcset/-/parse-srcset-1.0.2.tgz", + "integrity": "sha512-/2qh0lav6CmI15FzA3i/2Bzk2zCgQhGMkvhOhKNcBVQ1ldgpbfiNTVslmooUmWJcADi1f1kIeynbDRVzNlfR6Q==" + }, + "parse-url": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/parse-url/-/parse-url-6.0.0.tgz", + "integrity": "sha512-cYyojeX7yIIwuJzledIHeLUBVJ6COVLeT4eF+2P6aKVzwvgKQPndCBv3+yQ7pcWjqToYwaligxzSYNNmGoMAvw==", + "requires": { + "is-ssh": "^1.3.0", + "normalize-url": "^6.1.0", + "parse-path": "^4.0.0", + "protocols": "^1.4.0" + } + }, + "parse5": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/parse5/-/parse5-6.0.1.tgz", + "integrity": "sha512-Ofn/CTFzRGTTxwpNEs9PP93gXShHcTq255nzRYSKe8AkVpZY7e1fpmTfOyoIvjP5HG7Z2ZM7VS9PPhQGW2pOpw==" + }, + "parse5-htmlparser2-tree-adapter": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/parse5-htmlparser2-tree-adapter/-/parse5-htmlparser2-tree-adapter-6.0.1.tgz", + "integrity": "sha512-qPuWvbLgvDGilKc5BoicRovlT4MtYT6JfJyBOMDsKoiT+GiuP5qyrPCnR9HcPECIJJmZh5jRndyNThnhhb/vlA==", + "requires": { + "parse5": "^6.0.1" + } + }, + "parseqs": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/parseqs/-/parseqs-0.0.6.tgz", + "integrity": "sha512-jeAGzMDbfSHHA091hr0r31eYfTig+29g3GKKE/PPbEQ65X0lmMwlEoqmhzu0iztID5uJpZsFlUPDP8ThPL7M8w==" + }, + "parseuri": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/parseuri/-/parseuri-0.0.6.tgz", + "integrity": "sha512-AUjen8sAkGgao7UyCX6Ahv0gIK2fABKmYjvP4xmy5JaKvcbTRueIqIPHLAfq30xJddqSE033IOMUSOMCcK3Sow==" + }, + "parseurl": { + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/parseurl/-/parseurl-1.3.3.tgz", + "integrity": "sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ==" + }, + "pascal-case": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/pascal-case/-/pascal-case-3.1.2.tgz", + "integrity": "sha512-uWlGT3YSnK9x3BQJaOdcZwrnV6hPpd8jFH1/ucpiLRPh/2zCVJKS19E4GvYHvaCcACn3foXZ0cLB9Wrx1KGe5g==", + "requires": { + "no-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "password-prompt": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/password-prompt/-/password-prompt-1.1.2.tgz", + "integrity": "sha512-bpuBhROdrhuN3E7G/koAju0WjVw9/uQOG5Co5mokNj0MiOSBVZS1JTwM4zl55hu0WFmIEFvO9cU9sJQiBIYeIA==", + "requires": { + "ansi-escapes": "^3.1.0", + "cross-spawn": "^6.0.5" + } + }, + "path-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/path-case/-/path-case-3.0.4.tgz", + "integrity": "sha512-qO4qCFjXqVTrcbPt/hQfhTQ+VhFsqNKOPtytgNKkKxSoEp3XPUQ8ObFuePylOIok5gjn69ry8XiULxCwot3Wfg==", + "requires": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "path-exists": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", + "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==" + }, + "path-is-absolute": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", + "integrity": "sha1-F0uSaHNVNP+8es5r9TpanhtcX18=" + }, + "path-key": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-2.0.1.tgz", + "integrity": "sha1-QRyttXTFoUDTpLGRDUDYDMn0C0A=" + }, + "path-parse": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.7.tgz", + "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==" + }, + "path-root": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/path-root/-/path-root-0.1.1.tgz", + "integrity": "sha1-mkpoFMrBwM1zNgqV8yCDyOpHRbc=", + "requires": { + "path-root-regex": "^0.1.0" + } + }, + "path-root-regex": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/path-root-regex/-/path-root-regex-0.1.2.tgz", + "integrity": "sha1-v8zcjfWxLcUsi0PsONGNcsBLqW0=" + }, + "path-to-regexp": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-0.1.7.tgz", + "integrity": "sha1-32BBeABfUi8V60SQ5yR6G/qmf4w=" + }, + "path-type": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", + "integrity": "sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==" + }, + "peek-readable": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/peek-readable/-/peek-readable-4.1.0.tgz", + "integrity": "sha512-ZI3LnwUv5nOGbQzD9c2iDG6toheuXSZP5esSHBjopsXH4dg19soufvpUGA3uohi5anFtGb2lhAVdHzH6R/Evvg==" + }, + "performance-now": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/performance-now/-/performance-now-2.1.0.tgz", + "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==" + }, + "phin": { + "version": "2.9.3", + "resolved": "https://registry.npmjs.org/phin/-/phin-2.9.3.tgz", + "integrity": "sha512-CzFr90qM24ju5f88quFC/6qohjC144rehe5n6DH900lgXmUe86+xCKc10ev56gRKC4/BkHUoG4uSiQgBiIXwDA==" + }, + "physical-cpu-count": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/physical-cpu-count/-/physical-cpu-count-2.0.0.tgz", + "integrity": "sha1-GN4vl+S/epVRrXURlCtUlverpmA=" + }, + "picocolors": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.0.tgz", + "integrity": "sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ==" + }, + "picomatch": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", + "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==" + }, + "pixelmatch": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/pixelmatch/-/pixelmatch-4.0.2.tgz", + "integrity": "sha1-j0fc7FARtHe2fbA8JDvB8wheiFQ=", + "requires": { + "pngjs": "^3.0.0" + } + }, + "pkg-dir": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/pkg-dir/-/pkg-dir-4.2.0.tgz", + "integrity": "sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ==", + "requires": { + "find-up": "^4.0.0" + } + }, + "pkg-up": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/pkg-up/-/pkg-up-3.1.0.tgz", + "integrity": "sha512-nDywThFk1i4BQK4twPQ6TA4RT8bDY96yeuCVBWL3ePARCiEKDRSrNGbFIgUJpLp+XeIR65v8ra7WuJOFUBtkMA==", + "requires": { + "find-up": "^3.0.0" + }, + "dependencies": { + "find-up": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-3.0.0.tgz", + "integrity": "sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg==", + "requires": { + "locate-path": "^3.0.0" + } + }, + "locate-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-3.0.0.tgz", + "integrity": "sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A==", + "requires": { + "p-locate": "^3.0.0", + "path-exists": "^3.0.0" + } + }, + "p-locate": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-3.0.0.tgz", + "integrity": "sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ==", + "requires": { + "p-limit": "^2.0.0" + } + }, + "path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha1-zg6+ql94yxiSXqfYENe1mwEP1RU=" + } + } + }, + "platform": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/platform/-/platform-1.3.6.tgz", + "integrity": "sha512-fnWVljUchTro6RiCFvCXBbNhJc2NijN7oIQxbwsyL0buWJPG85v81ehlHI9fXrJsMNgTofEoWIQeClKpgxFLrg==" + }, + "pngjs": { + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/pngjs/-/pngjs-3.4.0.tgz", + "integrity": "sha512-NCrCHhWmnQklfH4MtJMRjZ2a8c80qXeMlQMv2uVp9ISJMTt562SbGd6n2oq0PaPgKm7Z6pL9E2UlLIhC+SHL3w==" + }, + "postcss": { + "version": "8.4.13", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-8.4.13.tgz", + "integrity": "sha512-jtL6eTBrza5MPzy8oJLFuUscHDXTV5KcLlqAWHl5q5WYRfnNRGSmOZmOZ1T6Gy7A99mOZfqungmZMpMmCVJ8ZA==", + "requires": { + "nanoid": "^3.3.3", + "picocolors": "^1.0.0", + "source-map-js": "^1.0.2" + } + }, + "postcss-calc": { + "version": "8.2.4", + "resolved": "https://registry.npmjs.org/postcss-calc/-/postcss-calc-8.2.4.tgz", + "integrity": "sha512-SmWMSJmB8MRnnULldx0lQIyhSNvuDl9HfrZkaqqE/WHAhToYsAvDq+yAsA/kIyINDszOp3Rh0GFoNuH5Ypsm3Q==", + "requires": { + "postcss-selector-parser": "^6.0.9", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-colormin": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/postcss-colormin/-/postcss-colormin-5.3.0.tgz", + "integrity": "sha512-WdDO4gOFG2Z8n4P8TWBpshnL3JpmNmJwdnfP2gbk2qBA8PWwOYcmjmI/t3CmMeL72a7Hkd+x/Mg9O2/0rD54Pg==", + "requires": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0", + "colord": "^2.9.1", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-convert-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-convert-values/-/postcss-convert-values-5.1.0.tgz", + "integrity": "sha512-GkyPbZEYJiWtQB0KZ0X6qusqFHUepguBCNFi9t5JJc7I2OTXG7C0twbTLvCfaKOLl3rSXmpAwV7W5txd91V84g==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-discard-comments": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-discard-comments/-/postcss-discard-comments-5.1.1.tgz", + "integrity": "sha512-5JscyFmvkUxz/5/+TB3QTTT9Gi9jHkcn8dcmmuN68JQcv3aQg4y88yEHHhwFB52l/NkaJ43O0dbksGMAo49nfQ==", + "requires": {} + }, + "postcss-discard-duplicates": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-duplicates/-/postcss-discard-duplicates-5.1.0.tgz", + "integrity": "sha512-zmX3IoSI2aoenxHV6C7plngHWWhUOV3sP1T8y2ifzxzbtnuhk1EdPwm0S1bIUNaJ2eNbWeGLEwzw8huPD67aQw==", + "requires": {} + }, + "postcss-discard-empty": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-discard-empty/-/postcss-discard-empty-5.1.1.tgz", + "integrity": "sha512-zPz4WljiSuLWsI0ir4Mcnr4qQQ5e1Ukc3i7UfE2XcrwKK2LIPIqE5jxMRxO6GbI3cv//ztXDsXwEWT3BHOGh3A==", + "requires": {} + }, + "postcss-discard-overridden": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-overridden/-/postcss-discard-overridden-5.1.0.tgz", + "integrity": "sha512-21nOL7RqWR1kasIVdKs8HNqQJhFxLsyRfAnUDm4Fe4t4mCWL9OJiHvlHPjcd8zc5Myu89b/7wZDnOSjFgeWRtw==", + "requires": {} + }, + "postcss-flexbugs-fixes": { + "version": "5.0.2", + "resolved": "https://registry.npmjs.org/postcss-flexbugs-fixes/-/postcss-flexbugs-fixes-5.0.2.tgz", + "integrity": "sha512-18f9voByak7bTktR2QgDveglpn9DTbBWPUzSOe9g0N4WR/2eSt6Vrcbf0hmspvMI6YWGywz6B9f7jzpFNJJgnQ==", + "requires": {} + }, + "postcss-loader": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/postcss-loader/-/postcss-loader-5.3.0.tgz", + "integrity": "sha512-/+Z1RAmssdiSLgIZwnJHwBMnlABPgF7giYzTN2NOfr9D21IJZ4mQC1R2miwp80zno9M4zMD/umGI8cR+2EL5zw==", + "requires": { + "cosmiconfig": "^7.0.0", + "klona": "^2.0.4", + "semver": "^7.3.4" + } + }, + "postcss-merge-longhand": { + "version": "5.1.4", + "resolved": "https://registry.npmjs.org/postcss-merge-longhand/-/postcss-merge-longhand-5.1.4.tgz", + "integrity": "sha512-hbqRRqYfmXoGpzYKeW0/NCZhvNyQIlQeWVSao5iKWdyx7skLvCfQFGIUsP9NUs3dSbPac2IC4Go85/zG+7MlmA==", + "requires": { + "postcss-value-parser": "^4.2.0", + "stylehacks": "^5.1.0" + } + }, + "postcss-merge-rules": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-merge-rules/-/postcss-merge-rules-5.1.1.tgz", + "integrity": "sha512-8wv8q2cXjEuCcgpIB1Xx1pIy8/rhMPIQqYKNzEdyx37m6gpq83mQQdCxgIkFgliyEnKvdwJf/C61vN4tQDq4Ww==", + "requires": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0", + "cssnano-utils": "^3.1.0", + "postcss-selector-parser": "^6.0.5" + } + }, + "postcss-minify-font-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-minify-font-values/-/postcss-minify-font-values-5.1.0.tgz", + "integrity": "sha512-el3mYTgx13ZAPPirSVsHqFzl+BBBDrXvbySvPGFnQcTI4iNslrPaFq4muTkLZmKlGk4gyFAYUBMH30+HurREyA==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-minify-gradients": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-minify-gradients/-/postcss-minify-gradients-5.1.1.tgz", + "integrity": "sha512-VGvXMTpCEo4qHTNSa9A0a3D+dxGFZCYwR6Jokk+/3oB6flu2/PnPXAh2x7x52EkY5xlIHLm+Le8tJxe/7TNhzw==", + "requires": { + "colord": "^2.9.1", + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-minify-params": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/postcss-minify-params/-/postcss-minify-params-5.1.2.tgz", + "integrity": "sha512-aEP+p71S/urY48HWaRHasyx4WHQJyOYaKpQ6eXl8k0kxg66Wt/30VR6/woh8THgcpRbonJD5IeD+CzNhPi1L8g==", + "requires": { + "browserslist": "^4.16.6", + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-minify-selectors": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/postcss-minify-selectors/-/postcss-minify-selectors-5.2.0.tgz", + "integrity": "sha512-vYxvHkW+iULstA+ctVNx0VoRAR4THQQRkG77o0oa4/mBS0OzGvvzLIvHDv/nNEM0crzN2WIyFU5X7wZhaUK3RA==", + "requires": { + "postcss-selector-parser": "^6.0.5" + } + }, + "postcss-modules-extract-imports": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-extract-imports/-/postcss-modules-extract-imports-3.0.0.tgz", + "integrity": "sha512-bdHleFnP3kZ4NYDhuGlVK+CMrQ/pqUm8bx/oGL93K6gVwiclvX5x0n76fYMKuIGKzlABOy13zsvqjb0f92TEXw==", + "requires": {} + }, + "postcss-modules-local-by-default": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.0.tgz", + "integrity": "sha512-sT7ihtmGSF9yhm6ggikHdV0hlziDTX7oFoXtuVWeDd3hHObNkcHRo9V3yg7vCAY7cONyxJC/XXCmmiHHcvX7bQ==", + "requires": { + "icss-utils": "^5.0.0", + "postcss-selector-parser": "^6.0.2", + "postcss-value-parser": "^4.1.0" + } + }, + "postcss-modules-scope": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz", + "integrity": "sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg==", + "requires": { + "postcss-selector-parser": "^6.0.4" + } + }, + "postcss-modules-values": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/postcss-modules-values/-/postcss-modules-values-4.0.0.tgz", + "integrity": "sha512-RDxHkAiEGI78gS2ofyvCsu7iycRv7oqw5xMWn9iMoR0N/7mf9D50ecQqUo5BZ9Zh2vH4bCUR/ktCqbB9m8vJjQ==", + "requires": { + "icss-utils": "^5.0.0" + } + }, + "postcss-normalize-charset": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-charset/-/postcss-normalize-charset-5.1.0.tgz", + "integrity": "sha512-mSgUJ+pd/ldRGVx26p2wz9dNZ7ji6Pn8VWBajMXFf8jk7vUoSrZ2lt/wZR7DtlZYKesmZI680qjr2CeFF2fbUg==", + "requires": {} + }, + "postcss-normalize-display-values": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-display-values/-/postcss-normalize-display-values-5.1.0.tgz", + "integrity": "sha512-WP4KIM4o2dazQXWmFaqMmcvsKmhdINFblgSeRgn8BJ6vxaMyaJkwAzpPpuvSIoG/rmX3M+IrRZEz2H0glrQNEA==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-positions": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-positions/-/postcss-normalize-positions-5.1.0.tgz", + "integrity": "sha512-8gmItgA4H5xiUxgN/3TVvXRoJxkAWLW6f/KKhdsH03atg0cB8ilXnrB5PpSshwVu/dD2ZsRFQcR1OEmSBDAgcQ==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-repeat-style": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-repeat-style/-/postcss-normalize-repeat-style-5.1.0.tgz", + "integrity": "sha512-IR3uBjc+7mcWGL6CtniKNQ4Rr5fTxwkaDHwMBDGGs1x9IVRkYIT/M4NelZWkAOBdV6v3Z9S46zqaKGlyzHSchw==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-string": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-string/-/postcss-normalize-string-5.1.0.tgz", + "integrity": "sha512-oYiIJOf4T9T1N4i+abeIc7Vgm/xPCGih4bZz5Nm0/ARVJ7K6xrDlLwvwqOydvyL3RHNf8qZk6vo3aatiw/go3w==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-timing-functions": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-timing-functions/-/postcss-normalize-timing-functions-5.1.0.tgz", + "integrity": "sha512-DOEkzJ4SAXv5xkHl0Wa9cZLF3WCBhF3o1SKVxKQAa+0pYKlueTpCgvkFAHfk+Y64ezX9+nITGrDZeVGgITJXjg==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-unicode": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-unicode/-/postcss-normalize-unicode-5.1.0.tgz", + "integrity": "sha512-J6M3MizAAZ2dOdSjy2caayJLQT8E8K9XjLce8AUQMwOrCvjCHv24aLC/Lps1R1ylOfol5VIDMaM/Lo9NGlk1SQ==", + "requires": { + "browserslist": "^4.16.6", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-url": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-normalize-url/-/postcss-normalize-url-5.1.0.tgz", + "integrity": "sha512-5upGeDO+PVthOxSmds43ZeMeZfKH+/DKgGRD7TElkkyS46JXAUhMzIKiCa7BabPeIy3AQcTkXwVVN7DbqsiCew==", + "requires": { + "normalize-url": "^6.0.1", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-normalize-whitespace": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-normalize-whitespace/-/postcss-normalize-whitespace-5.1.1.tgz", + "integrity": "sha512-83ZJ4t3NUDETIHTa3uEg6asWjSBYL5EdkVB0sDncx9ERzOKBVJIUeDO9RyA9Zwtig8El1d79HBp0JEi8wvGQnA==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-ordered-values": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-ordered-values/-/postcss-ordered-values-5.1.1.tgz", + "integrity": "sha512-7lxgXF0NaoMIgyihL/2boNAEZKiW0+HkMhdKMTD93CjW8TdCy2hSdj8lsAo+uwm7EDG16Da2Jdmtqpedl0cMfw==", + "requires": { + "cssnano-utils": "^3.1.0", + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-reduce-initial": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-reduce-initial/-/postcss-reduce-initial-5.1.0.tgz", + "integrity": "sha512-5OgTUviz0aeH6MtBjHfbr57tml13PuedK/Ecg8szzd4XRMbYxH4572JFG067z+FqBIf6Zp/d+0581glkvvWMFw==", + "requires": { + "browserslist": "^4.16.6", + "caniuse-api": "^3.0.0" + } + }, + "postcss-reduce-transforms": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-reduce-transforms/-/postcss-reduce-transforms-5.1.0.tgz", + "integrity": "sha512-2fbdbmgir5AvpW9RLtdONx1QoYG2/EtqpNQbFASDlixBbAYuTcJ0dECwlqNqH7VbaUnEnh8SrxOe2sRIn24XyQ==", + "requires": { + "postcss-value-parser": "^4.2.0" + } + }, + "postcss-selector-parser": { + "version": "6.0.10", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-6.0.10.tgz", + "integrity": "sha512-IQ7TZdoaqbT+LCpShg46jnZVlhWD2w6iQYAcYXfHARZ7X1t/UGhhceQDs5X0cGqKvYlHNOuv7Oa1xmb0oQuA3w==", + "requires": { + "cssesc": "^3.0.0", + "util-deprecate": "^1.0.2" + } + }, + "postcss-svgo": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/postcss-svgo/-/postcss-svgo-5.1.0.tgz", + "integrity": "sha512-D75KsH1zm5ZrHyxPakAxJWtkyXew5qwS70v56exwvw542d9CRtTo78K0WeFxZB4G7JXKKMbEZtZayTGdIky/eA==", + "requires": { + "postcss-value-parser": "^4.2.0", + "svgo": "^2.7.0" + } + }, + "postcss-unique-selectors": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/postcss-unique-selectors/-/postcss-unique-selectors-5.1.1.tgz", + "integrity": "sha512-5JiODlELrz8L2HwxfPnhOWZYWDxVHWL83ufOv84NrcgipI7TaeRsatAhK4Tr2/ZiYldpK/wBvw5BD3qfaK96GA==", + "requires": { + "postcss-selector-parser": "^6.0.5" + } + }, + "postcss-value-parser": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/postcss-value-parser/-/postcss-value-parser-4.2.0.tgz", + "integrity": "sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ==" + }, + "prebuild-install": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/prebuild-install/-/prebuild-install-7.1.0.tgz", + "integrity": "sha512-CNcMgI1xBypOyGqjp3wOc8AAo1nMhZS3Cwd3iHIxOdAUbb+YxdNuM4Z5iIrZ8RLvOsf3F3bl7b7xGq6DjQoNYA==", + "requires": { + "detect-libc": "^2.0.0", + "expand-template": "^2.0.3", + "github-from-package": "0.0.0", + "minimist": "^1.2.3", + "mkdirp-classic": "^0.5.3", + "napi-build-utils": "^1.0.1", + "node-abi": "^3.3.0", + "npmlog": "^4.0.1", + "pump": "^3.0.0", + "rc": "^1.2.7", + "simple-get": "^4.0.0", + "tar-fs": "^2.0.0", + "tunnel-agent": "^0.6.0" + }, + "dependencies": { + "detect-libc": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.1.tgz", + "integrity": "sha512-463v3ZeIrcWtdgIg6vI6XUncguvr2TnGl4SzDXinkt9mSLpBJKXT3mW6xT3VQdDN11+WVs29pgvivTc4Lp8v+w==" + } + } + }, + "prelude-ls": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", + "integrity": "sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g==" + }, + "prepend-http": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/prepend-http/-/prepend-http-2.0.0.tgz", + "integrity": "sha1-6SQ0v6XqjBn0HN/UAddBo8gZ2Jc=" + }, + "prettier": { + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.6.2.tgz", + "integrity": "sha512-PkUpF+qoXTqhOeWL9fu7As8LXsIUZ1WYaJiY/a7McAQzxjk82OF0tibkFXVCDImZtWxbvojFjerkiLb0/q8mew==", + "dev": true + }, + "pretty-bytes": { + "version": "5.6.0", + "resolved": "https://registry.npmjs.org/pretty-bytes/-/pretty-bytes-5.6.0.tgz", + "integrity": "sha512-FFw039TmrBqFK8ma/7OL3sDz/VytdtJr044/QUJtH0wK9lb9jLq9tJyIxUwtQJHwar2BqtiA4iCWSwo9JLkzFg==" + }, + "pretty-error": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/pretty-error/-/pretty-error-2.1.2.tgz", + "integrity": "sha512-EY5oDzmsX5wvuynAByrmY0P0hcp+QpnAKbJng2A2MPjVKXCxrDSUkzghVJ4ZGPIv+JC4gX8fPUWscC0RtjsWGw==", + "requires": { + "lodash": "^4.17.20", + "renderkid": "^2.0.4" + } + }, + "probe-image-size": { + "version": "7.2.3", + "resolved": "https://registry.npmjs.org/probe-image-size/-/probe-image-size-7.2.3.tgz", + "integrity": "sha512-HubhG4Rb2UH8YtV4ba0Vp5bQ7L78RTONYu/ujmCu5nBI8wGv24s4E9xSKBi0N1MowRpxk76pFCpJtW0KPzOK0w==", + "requires": { + "lodash.merge": "^4.6.2", + "needle": "^2.5.2", + "stream-parser": "~0.3.1" + } + }, + "process": { + "version": "0.11.10", + "resolved": "https://registry.npmjs.org/process/-/process-0.11.10.tgz", + "integrity": "sha1-czIwDoQBYb2j5podHZGn1LwW8YI=" + }, + "process-nextick-args": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/process-nextick-args/-/process-nextick-args-2.0.1.tgz", + "integrity": "sha512-3ouUOpQhtgrbOa17J7+uxOTpITYWaGP7/AhoR3+A+/1e9skrzelGi/dXzEYyvbxubEF6Wn2ypscTKiKJFFn1ag==" + }, + "progress": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/progress/-/progress-2.0.3.tgz", + "integrity": "sha512-7PiHtLll5LdnKIMw100I+8xJXR5gW2QwWYkT6iJva0bXitZKa/XMrSbdmg3r2Xnaidz9Qumd0VPaMrZlF9V9sA==" + }, + "promise": { + "version": "7.3.1", + "resolved": "https://registry.npmjs.org/promise/-/promise-7.3.1.tgz", + "integrity": "sha512-nolQXZ/4L+bP/UGlkfaIujX9BKxGwmQ9OT4mOt5yvy8iK1h3wqTEJCijzGANTCCl9nWjY41juyAn2K3Q1hLLTg==", + "requires": { + "asap": "~2.0.3" + } + }, + "prompts": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/prompts/-/prompts-2.4.2.tgz", + "integrity": "sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q==", + "requires": { + "kleur": "^3.0.3", + "sisteransi": "^1.0.5" + } + }, + "prop-types": { + "version": "15.8.1", + "resolved": "https://registry.npmjs.org/prop-types/-/prop-types-15.8.1.tgz", + "integrity": "sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg==", + "requires": { + "loose-envify": "^1.4.0", + "object-assign": "^4.1.1", + "react-is": "^16.13.1" + } + }, + "proper-lockfile": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/proper-lockfile/-/proper-lockfile-4.1.2.tgz", + "integrity": "sha512-TjNPblN4BwAWMXU8s9AEz4JmQxnD1NNL7bNOY/AKUzyamc379FWASUhc/K1pL2noVb+XmZKLL68cjzLsiOAMaA==", + "requires": { + "graceful-fs": "^4.2.4", + "retry": "^0.12.0", + "signal-exit": "^3.0.2" + } + }, + "property-information": { + "version": "5.6.0", + "resolved": "https://registry.npmmirror.com/property-information/-/property-information-5.6.0.tgz", + "integrity": "sha512-YUHSPk+A30YPv+0Qf8i9Mbfe/C0hdPXk1s1jPVToV8pk8BQtpw10ct89Eo7OWkutrwqvT0eicAxlOg3dOAu8JA==", + "requires": { + "xtend": "^4.0.0" + } + }, + "protocols": { + "version": "1.4.8", + "resolved": "https://registry.npmjs.org/protocols/-/protocols-1.4.8.tgz", + "integrity": "sha512-IgjKyaUSjsROSO8/D49Ab7hP8mJgTYcqApOqdPhLoPxAplXmkp+zRvsrSQjFn5by0rhm4VH0GAUELIPpx7B1yg==" + }, + "proxy-addr": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/proxy-addr/-/proxy-addr-2.0.7.tgz", + "integrity": "sha512-llQsMLSUDUPT44jdrU/O37qlnifitDP+ZwrmmZcoSKyLKvtZxpyV0n2/bD/N4tBAAZ/gJEdZU7KMraoK1+XYAg==", + "requires": { + "forwarded": "0.2.0", + "ipaddr.js": "1.9.1" + } + }, + "pseudomap": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/pseudomap/-/pseudomap-1.0.2.tgz", + "integrity": "sha1-8FKijacOYYkX7wqKw0wa5aaChrM=" + }, + "psl": { + "version": "1.8.0", + "resolved": "https://registry.npmmirror.com/psl/-/psl-1.8.0.tgz", + "integrity": "sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ==" + }, + "pump": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/pump/-/pump-3.0.0.tgz", + "integrity": "sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww==", + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "punycode": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.1.1.tgz", + "integrity": "sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A==" + }, + "pupa": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/pupa/-/pupa-2.1.1.tgz", + "integrity": "sha512-l1jNAspIBSFqbT+y+5FosojNpVpF94nlI+wDUpqP9enwOTfHx9f0gh5nB96vl+6yTpsJsypeNrwfzPrKuHB41A==", + "requires": { + "escape-goat": "^2.0.0" + } + }, + "q": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/q/-/q-1.5.1.tgz", + "integrity": "sha1-fjL3W0E4EpHQRhHxvxQQmsAGUdc=" + }, + "qs": { + "version": "6.10.3", + "resolved": "https://registry.npmjs.org/qs/-/qs-6.10.3.tgz", + "integrity": "sha512-wr7M2E0OFRfIfJZjKGieI8lBKb7fRCH4Fv5KNPEs7gJ8jadvotdsS08PzOKR7opXhZ/Xkjtt3WF9g38drmyRqQ==", + "requires": { + "side-channel": "^1.0.4" + } + }, + "query-string": { + "version": "6.14.1", + "resolved": "https://registry.npmjs.org/query-string/-/query-string-6.14.1.tgz", + "integrity": "sha512-XDxAeVmpfu1/6IjyT/gXHOl+S0vQ9owggJ30hhWKdHAsNPOcasn5o9BW0eejZqL2e4vMjhAxoW3jVHcD6mbcYw==", + "requires": { + "decode-uri-component": "^0.2.0", + "filter-obj": "^1.1.0", + "split-on-first": "^1.0.0", + "strict-uri-encode": "^2.0.0" + } + }, + "querystring": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/querystring/-/querystring-0.2.1.tgz", + "integrity": "sha512-wkvS7mL/JMugcup3/rMitHmd9ecIGd2lhFhK9N3UUQ450h66d1r3Y9nvXzQAW1Lq+wyx61k/1pfKS5KuKiyEbg==" + }, + "queue-microtask": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/queue-microtask/-/queue-microtask-1.2.3.tgz", + "integrity": "sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==" + }, + "quick-lru": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/quick-lru/-/quick-lru-5.1.1.tgz", + "integrity": "sha512-WuyALRjWPDGtt/wzJiadO5AXY+8hZ80hVpe6MyivgraREW751X3SbhRvG3eLKOYN+8VEvqLcf3wdnt44Z4S4SA==" + }, + "randombytes": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", + "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", + "requires": { + "safe-buffer": "^5.1.0" + } + }, + "range-parser": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/range-parser/-/range-parser-1.2.1.tgz", + "integrity": "sha512-Hrgsx+orqoygnmhFbKaHE6c296J+HTAQXoxEF6gNupROmmGJRoyzfG3ccAveqCBrwr/2yxQ5BVd/GTl5agOwSg==" + }, + "raw-body": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/raw-body/-/raw-body-2.5.1.tgz", + "integrity": "sha512-qqJBtEyVgS0ZmPGdCFPWJ3FreoqvG4MVQln/kCgF7Olq95IbOp0/BWyMwbdtn4VTvkM8Y7khCQ2Xgk/tcrCXig==", + "requires": { + "bytes": "3.1.2", + "http-errors": "2.0.0", + "iconv-lite": "0.4.24", + "unpipe": "1.0.0" + } + }, + "raw-loader": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/raw-loader/-/raw-loader-4.0.2.tgz", + "integrity": "sha512-ZnScIV3ag9A4wPX/ZayxL/jZH+euYb6FcUinPcgiQW0+UBtEv0O6Q3lGd3cqJ+GHH+rksEv3Pj99oxJ3u3VIKA==", + "requires": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "rc": { + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/rc/-/rc-1.2.8.tgz", + "integrity": "sha512-y3bGgqKj3QBdxLbLkomlohkvsA8gdAiUQlSBJnBhfn+BPxg4bc62d8TcBW15wavDfgexCgccckhcZvywyQYPOw==", + "requires": { + "deep-extend": "^0.6.0", + "ini": "~1.3.0", + "minimist": "^1.2.0", + "strip-json-comments": "~2.0.1" + }, + "dependencies": { + "ini": { + "version": "1.3.8", + "resolved": "https://registry.npmjs.org/ini/-/ini-1.3.8.tgz", + "integrity": "sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==" + }, + "strip-json-comments": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-2.0.1.tgz", + "integrity": "sha1-PFMZQukIwml8DsNEhYwobHygpgo=" + } + } + }, + "react": { + "version": "17.0.1", + "integrity": "sha512-lG9c9UuMHdcAexXtigOZLX8exLWkW0Ku29qPRU8uhF2R9BN96dLCt0psvzPLlHc5OWkgymP3qwTRgbnw5BKx3w==", + "requires": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1" + } + }, + "react-dev-utils": { + "version": "12.0.1", + "resolved": "https://registry.npmjs.org/react-dev-utils/-/react-dev-utils-12.0.1.tgz", + "integrity": "sha512-84Ivxmr17KjUupyqzFode6xKhjwuEJDROWKJy/BthkL7Wn6NJ8h4WE6k/exAv6ImS+0oZLRRW5j/aINMHyeGeQ==", + "requires": { + "@babel/code-frame": "^7.16.0", + "address": "^1.1.2", + "browserslist": "^4.18.1", + "chalk": "^4.1.2", + "cross-spawn": "^7.0.3", + "detect-port-alt": "^1.1.6", + "escape-string-regexp": "^4.0.0", + "filesize": "^8.0.6", + "find-up": "^5.0.0", + "fork-ts-checker-webpack-plugin": "^6.5.0", + "global-modules": "^2.0.0", + "globby": "^11.0.4", + "gzip-size": "^6.0.0", + "immer": "^9.0.7", + "is-root": "^2.1.0", + "loader-utils": "^3.2.0", + "open": "^8.4.0", + "pkg-up": "^3.1.0", + "prompts": "^2.4.2", + "react-error-overlay": "^6.0.11", + "recursive-readdir": "^2.2.2", + "shell-quote": "^1.7.3", + "strip-ansi": "^6.0.1", + "text-table": "^0.2.0" + }, + "dependencies": { + "cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "requires": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + } + }, + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "detect-port-alt": { + "version": "1.1.6", + "resolved": "https://registry.npmjs.org/detect-port-alt/-/detect-port-alt-1.1.6.tgz", + "integrity": "sha512-5tQykt+LqfJFBEYaDITx7S7cR7mJ/zQmLXZ2qt5w04ainYZw6tBf9dBunMjVeVOdYVRUzUOE4HkY5J7+uttb5Q==", + "requires": { + "address": "^1.0.1", + "debug": "^2.6.0" + } + }, + "escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==" + }, + "find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "requires": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + } + }, + "loader-utils": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-3.2.0.tgz", + "integrity": "sha512-HVl9ZqccQihZ7JM85dco1MvO9G+ONvxoGa9rkhzFsneGLKSUg1gJf9bWzhRhcvm2qChhWpebQhP44qxjKIUCaQ==" + }, + "locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "requires": { + "p-locate": "^5.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + }, + "open": { + "version": "8.4.0", + "resolved": "https://registry.npmjs.org/open/-/open-8.4.0.tgz", + "integrity": "sha512-XgFPPM+B28FtCCgSb9I+s9szOC1vZRSwgWsRUA5ylIxRTgKozqjOCrVOqGsYABPYK5qnfqClxZTFBa8PKt2v6Q==", + "requires": { + "define-lazy-prop": "^2.0.0", + "is-docker": "^2.1.1", + "is-wsl": "^2.2.0" + } + }, + "p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "requires": { + "yocto-queue": "^0.1.0" + } + }, + "p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "requires": { + "p-limit": "^3.0.2" + } + }, + "path-key": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", + "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==" + }, + "shebang-command": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", + "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "requires": { + "shebang-regex": "^3.0.0" + } + }, + "shebang-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", + "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==" + }, + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "requires": { + "isexe": "^2.0.0" + } + } + } + }, + "react-dom": { + "version": "17.0.1", + "integrity": "sha512-6eV150oJZ9U2t9svnsspTMrWNyHc6chX0KzDeAOXftRa8bNeOKTTfCJ7KorIwenkHd2xqVTBTCZd79yk/lx/Ug==", + "requires": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1", + "scheduler": "^0.20.1" + } + }, + "react-error-overlay": { + "version": "6.0.11", + "resolved": "https://registry.npmjs.org/react-error-overlay/-/react-error-overlay-6.0.11.tgz", + "integrity": "sha512-/6UZ2qgEyH2aqzYZgQPxEnz33NJ2gNsnHA2o5+o4wW9bLM/JYQitNP9xPhsXwC08hMMovfGe/8retsdDsczPRg==" + }, + "react-fast-compare": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/react-fast-compare/-/react-fast-compare-3.2.0.tgz", + "integrity": "sha512-rtGImPZ0YyLrscKI9xTpV8psd6I8VAtjKCzQDlzyDvqJA8XOW78TXYQwNRNd8g8JZnDu8q9Fu/1v4HPAVwVdHA==" + }, + "react-helmet": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/react-helmet/-/react-helmet-6.1.0.tgz", + "integrity": "sha512-4uMzEY9nlDlgxr61NL3XbKRy1hEkXmKNXhjbAIOVw5vcFrsdYbH2FEwcNyWvWinl103nXgzYNlns9ca+8kFiWw==", + "requires": { + "object-assign": "^4.1.1", + "prop-types": "^15.7.2", + "react-fast-compare": "^3.1.1", + "react-side-effect": "^2.1.0" + } + }, + "react-is": { + "version": "16.13.1", + "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.13.1.tgz", + "integrity": "sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==" + }, + "react-lifecycles-compat": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz", + "integrity": "sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA==" + }, + "react-popper": { + "version": "2.3.0", + "resolved": "https://registry.npmmirror.com/react-popper/-/react-popper-2.3.0.tgz", + "integrity": "sha512-e1hj8lL3uM+sgSR4Lxzn5h1GxBlpa4CQz0XLF8kx4MDrDRWY0Ena4c97PUeSX9i5W3UAfDP0z0FXCTQkoXUl3Q==", + "requires": { + "react-fast-compare": "^3.0.1", + "warning": "^4.0.2" + } + }, + "react-property": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/react-property/-/react-property-2.0.0.tgz", + "integrity": "sha512-kzmNjIgU32mO4mmH5+iUyrqlpFQhF8K2k7eZ4fdLSOPFrD1XgEuSBv9LDEgxRXTMBqMd8ppT0x6TIzqE5pdGdw==" + }, + "react-refresh": { + "version": "0.9.0", + "resolved": "https://registry.npmjs.org/react-refresh/-/react-refresh-0.9.0.tgz", + "integrity": "sha512-Gvzk7OZpiqKSkxsQvO/mbTN1poglhmAV7gR/DdIrRrSMXraRQQlfikRJOr3Nb9GTMPC5kof948Zy6jJZIFtDvQ==" + }, + "react-side-effect": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/react-side-effect/-/react-side-effect-2.1.1.tgz", + "integrity": "sha512-2FoTQzRNTncBVtnzxFOk2mCpcfxQpenBMbk5kSVBg5UcPqV9fRbgY2zhb7GTWWOlpFmAxhClBDlIq8Rsubz1yQ==", + "requires": {} + }, + "react-textarea-autosize": { + "version": "8.3.3", + "resolved": "https://registry.npmmirror.com/react-textarea-autosize/-/react-textarea-autosize-8.3.3.tgz", + "integrity": "sha512-2XlHXK2TDxS6vbQaoPbMOfQ8GK7+irc2fVK6QFIcC8GOnH3zI/v481n+j1L0WaPVvKxwesnY93fEfH++sus2rQ==", + "requires": { + "@babel/runtime": "^7.10.2", + "use-composed-ref": "^1.0.0", + "use-latest": "^1.0.0" + } + }, + "read": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/read/-/read-1.0.7.tgz", + "integrity": "sha1-s9oZvQUkMal2cdRKQmNK33ELQMQ=", + "requires": { + "mute-stream": "~0.0.4" + } + }, + "read-pkg": { + "version": "5.2.0", + "resolved": "https://registry.npmmirror.com/read-pkg/-/read-pkg-5.2.0.tgz", + "integrity": "sha512-Ug69mNOpfvKDAc2Q8DRpMjjzdtrnv9HcSMX+4VsZxD1aZ6ZzrIE7rlzXBtWTyhULSMKg076AW6WR5iZpD0JiOg==", + "requires": { + "@types/normalize-package-data": "^2.4.0", + "normalize-package-data": "^2.5.0", + "parse-json": "^5.0.0", + "type-fest": "^0.6.0" + }, + "dependencies": { + "hosted-git-info": { + "version": "2.8.9", + "resolved": "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz", + "integrity": "sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw==" + }, + "normalize-package-data": { + "version": "2.5.0", + "resolved": "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-2.5.0.tgz", + "integrity": "sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA==", + "requires": { + "hosted-git-info": "^2.1.4", + "resolve": "^1.10.0", + "semver": "2 || 3 || 4 || 5", + "validate-npm-package-license": "^3.0.1" + } + }, + "semver": { + "version": "5.7.1", + "resolved": "https://registry.npmmirror.com/semver/-/semver-5.7.1.tgz", + "integrity": "sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ==" + }, + "type-fest": { + "version": "0.6.0", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.6.0.tgz", + "integrity": "sha512-q+MB8nYR1KDLrgr4G5yemftpMC7/QLqVndBmEEdqzmNj5dcFOO4Oo8qlwZE3ULT3+Zim1F8Kq4cBnikNhlCMlg==" + } + } + }, + "read-pkg-up": { + "version": "7.0.1", + "resolved": "https://registry.npmmirror.com/read-pkg-up/-/read-pkg-up-7.0.1.tgz", + "integrity": "sha512-zK0TB7Xd6JpCLmlLmufqykGE+/TlOePD6qKClNW7hHDKFh/J7/7gCWGR7joEQEW1bKq3a3yUZSObOoWLFQ4ohg==", + "requires": { + "find-up": "^4.1.0", + "read-pkg": "^5.2.0", + "type-fest": "^0.8.1" + }, + "dependencies": { + "type-fest": { + "version": "0.8.1", + "resolved": "https://registry.npmmirror.com/type-fest/-/type-fest-0.8.1.tgz", + "integrity": "sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA==" + } + } + }, + "readable-stream": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.0.tgz", + "integrity": "sha512-BViHy7LKeTz4oNnkcLJ+lVSL6vpiFeX6/d3oSH8zCW7UxP2onchk+vTGB143xuFjHS3deTgkKoXXymXqymiIdA==", + "requires": { + "inherits": "^2.0.3", + "string_decoder": "^1.1.1", + "util-deprecate": "^1.0.1" + } + }, + "readable-web-to-node-stream": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/readable-web-to-node-stream/-/readable-web-to-node-stream-3.0.2.tgz", + "integrity": "sha512-ePeK6cc1EcKLEhJFt/AebMCLL+GgSKhuygrZ/GLaKZYEecIgIECf4UaUuaByiGtzckwR4ain9VzUh95T1exYGw==", + "requires": { + "readable-stream": "^3.6.0" + } + }, + "readdirp": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz", + "integrity": "sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==", + "requires": { + "picomatch": "^2.2.1" + } + }, + "recursive-readdir": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/recursive-readdir/-/recursive-readdir-2.2.2.tgz", + "integrity": "sha512-nRCcW9Sj7NuZwa2XvH9co8NPeXUBhZP7CRKJtU+cS6PW9FpCIFoI5ib0NT1ZrbNuPoRy0ylyCaUL8Gih4LSyFg==", + "requires": { + "minimatch": "3.0.4" + }, + "dependencies": { + "minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "requires": { + "brace-expansion": "^1.1.7" + } + } + } + }, + "redent": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/redent/-/redent-3.0.0.tgz", + "integrity": "sha512-6tDA8g98We0zd0GvVeMT9arEOnTw9qM03L9cJXaCjrip1OO764RDBLBfrB4cwzNGDj5OA5ioymC9GkizgWJDUg==", + "requires": { + "indent-string": "^4.0.0", + "strip-indent": "^3.0.0" + } + }, + "redux": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/redux/-/redux-4.1.2.tgz", + "integrity": "sha512-SH8PglcebESbd/shgf6mii6EIoRM0zrQyjcuQ+ojmfxjTtE0z9Y8pa62iA/OJ58qjP6j27uyW4kUF4jl/jd6sw==", + "requires": { + "@babel/runtime": "^7.9.2" + } + }, + "redux-thunk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/redux-thunk/-/redux-thunk-2.4.1.tgz", + "integrity": "sha512-OOYGNY5Jy2TWvTL1KgAlVy6dcx3siPJ1wTq741EPyUKfn6W6nChdICjZwCd0p8AZBs5kWpZlbkXW2nE/zjUa+Q==", + "requires": {} + }, + "regenerate": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/regenerate/-/regenerate-1.4.2.tgz", + "integrity": "sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A==" + }, + "regenerate-unicode-properties": { + "version": "10.0.1", + "resolved": "https://registry.npmjs.org/regenerate-unicode-properties/-/regenerate-unicode-properties-10.0.1.tgz", + "integrity": "sha512-vn5DU6yg6h8hP/2OkQo3K7uVILvY4iu0oI4t3HFa81UPkhGJwkRwM10JEc3upjdhHjs/k8GJY1sRBhk5sr69Bw==", + "requires": { + "regenerate": "^1.4.2" + } + }, + "regenerator-runtime": { + "version": "0.13.9", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz", + "integrity": "sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA==" + }, + "regenerator-transform": { + "version": "0.15.0", + "resolved": "https://registry.npmjs.org/regenerator-transform/-/regenerator-transform-0.15.0.tgz", + "integrity": "sha512-LsrGtPmbYg19bcPHwdtmXwbW+TqNvtY4riE3P83foeHRroMbH6/2ddFBfab3t7kbzc7v7p4wbkIecHImqt0QNg==", + "requires": { + "@babel/runtime": "^7.8.4" + } + }, + "regex-parser": { + "version": "2.2.11", + "resolved": "https://registry.npmmirror.com/regex-parser/-/regex-parser-2.2.11.tgz", + "integrity": "sha512-jbD/FT0+9MBU2XAZluI7w2OBs1RBi6p9M83nkoZayQXXU9e8Robt69FcZc7wU4eJD/YFTjn1JdCk3rbMJajz8Q==" + }, + "regexp.prototype.flags": { + "version": "1.4.3", + "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.4.3.tgz", + "integrity": "sha512-fjggEOO3slI6Wvgjwflkc4NFRCTZAu5CnNfBd5qOMYhWdn67nJBBu34/TkD++eeFmd8C9r9jfXJ27+nSiRkSUA==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "functions-have-names": "^1.2.2" + } + }, + "regexpp": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/regexpp/-/regexpp-3.2.0.tgz", + "integrity": "sha512-pq2bWo9mVD43nbts2wGv17XLiNLya+GklZ8kaDLV2Z08gDCsGpnKn9BFMepvWuHCbyVvY7J5o5+BVvoQbmlJLg==" + }, + "regexpu-core": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/regexpu-core/-/regexpu-core-5.0.1.tgz", + "integrity": "sha512-CriEZlrKK9VJw/xQGJpQM5rY88BtuL8DM+AEwvcThHilbxiTAy8vq4iJnd2tqq8wLmjbGZzP7ZcKFjbGkmEFrw==", + "requires": { + "regenerate": "^1.4.2", + "regenerate-unicode-properties": "^10.0.1", + "regjsgen": "^0.6.0", + "regjsparser": "^0.8.2", + "unicode-match-property-ecmascript": "^2.0.0", + "unicode-match-property-value-ecmascript": "^2.0.0" + } + }, + "registry-auth-token": { + "version": "4.2.1", + "resolved": "https://registry.npmjs.org/registry-auth-token/-/registry-auth-token-4.2.1.tgz", + "integrity": "sha512-6gkSb4U6aWJB4SF2ZvLb76yCBjcvufXBqvvEx1HbmKPkutswjW1xNVRY0+daljIYRbogN7O0etYSlbiaEQyMyw==", + "requires": { + "rc": "^1.2.8" + } + }, + "registry-url": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/registry-url/-/registry-url-5.1.0.tgz", + "integrity": "sha512-8acYXXTI0AkQv6RAOjE3vOaIXZkT9wo4LOFbBKYQEEnnMNBpKqdUrI6S4NT0KPIo/WVvJ5tE/X5LF/TQUf0ekw==", + "requires": { + "rc": "^1.2.8" + } + }, + "regjsgen": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/regjsgen/-/regjsgen-0.6.0.tgz", + "integrity": "sha512-ozE883Uigtqj3bx7OhL1KNbCzGyW2NQZPl6Hs09WTvCuZD5sTI4JY58bkbQWa/Y9hxIsvJ3M8Nbf7j54IqeZbA==" + }, + "regjsparser": { + "version": "0.8.4", + "resolved": "https://registry.npmjs.org/regjsparser/-/regjsparser-0.8.4.tgz", + "integrity": "sha512-J3LABycON/VNEu3abOviqGHuB/LOtOQj8SKmfP9anY5GfAVw/SPjwzSjxGjbZXIxbGfqTHtJw58C2Li/WkStmA==", + "requires": { + "jsesc": "~0.5.0" + }, + "dependencies": { + "jsesc": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-0.5.0.tgz", + "integrity": "sha1-597mbjXW/Bb3EP6R1c9p9w8IkR0=" + } + } + }, + "relay-compiler": { + "version": "12.0.0", + "resolved": "https://registry.npmjs.org/relay-compiler/-/relay-compiler-12.0.0.tgz", + "integrity": "sha512-SWqeSQZ+AMU/Cr7iZsHi1e78Z7oh00I5SvR092iCJq79aupqJ6Ds+I1Pz/Vzo5uY5PY0jvC4rBJXzlIN5g9boQ==", + "requires": { + "@babel/core": "^7.14.0", + "@babel/generator": "^7.14.0", + "@babel/parser": "^7.14.0", + "@babel/runtime": "^7.0.0", + "@babel/traverse": "^7.14.0", + "@babel/types": "^7.0.0", + "babel-preset-fbjs": "^3.4.0", + "chalk": "^4.0.0", + "fb-watchman": "^2.0.0", + "fbjs": "^3.0.0", + "glob": "^7.1.1", + "immutable": "~3.7.6", + "invariant": "^2.2.4", + "nullthrows": "^1.1.1", + "relay-runtime": "12.0.0", + "signedsource": "^1.0.0", + "yargs": "^15.3.1" + } + }, + "relay-runtime": { + "version": "12.0.0", + "resolved": "https://registry.npmjs.org/relay-runtime/-/relay-runtime-12.0.0.tgz", + "integrity": "sha512-QU6JKr1tMsry22DXNy9Whsq5rmvwr3LSZiiWV/9+DFpuTWvp+WFhobWMc8TC4OjKFfNhEZy7mOiqUAn5atQtug==", + "requires": { + "@babel/runtime": "^7.0.0", + "fbjs": "^3.0.0", + "invariant": "^2.2.4" + } + }, + "remark": { + "version": "13.0.0", + "resolved": "https://registry.npmmirror.com/remark/-/remark-13.0.0.tgz", + "integrity": "sha512-HDz1+IKGtOyWN+QgBiAT0kn+2s6ovOxHyPAFGKVE81VSzJ+mq7RwHFledEvB5F1p4iJvOah/LOKdFuzvRnNLCA==", + "requires": { + "remark-parse": "^9.0.0", + "remark-stringify": "^9.0.0", + "unified": "^9.1.0" + } + }, + "remark-footnotes": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/remark-footnotes/-/remark-footnotes-3.0.0.tgz", + "integrity": "sha512-ZssAvH9FjGYlJ/PBVKdSmfyPc3Cz4rTWgZLI4iE/SX8Nt5l3o3oEjv3wwG5VD7xOjktzdwp5coac+kJV9l4jgg==", + "requires": { + "mdast-util-footnote": "^0.1.0", + "micromark-extension-footnote": "^0.3.0" + } + }, + "remark-gfm": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/remark-gfm/-/remark-gfm-1.0.0.tgz", + "integrity": "sha512-KfexHJCiqvrdBZVbQ6RopMZGwaXz6wFJEfByIuEwGf0arvITHjiKKZ1dpXujjH9KZdm1//XJQwgfnJ3lmXaDPA==", + "requires": { + "mdast-util-gfm": "^0.1.0", + "micromark-extension-gfm": "^0.3.0" + } + }, + "remark-parse": { + "version": "9.0.0", + "resolved": "https://registry.npmmirror.com/remark-parse/-/remark-parse-9.0.0.tgz", + "integrity": "sha512-geKatMwSzEXKHuzBNU1z676sGcDcFoChMK38TgdHJNAYfFtsfHDQG7MoJAjs6sgYMqyLduCYWDIWZIxiPeafEw==", + "requires": { + "mdast-util-from-markdown": "^0.8.0" + } + }, + "remark-retext": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/remark-retext/-/remark-retext-4.0.0.tgz", + "integrity": "sha512-cYCchalpf25bTtfXF24ribYvqytPKq0TiEhqQDBHvVEEsApebwruPWP1cTcvTFBidmpXyqzycm+y8ng7Kmvc8Q==", + "requires": { + "mdast-util-to-nlcst": "^4.0.0" + } + }, + "remark-stringify": { + "version": "9.0.1", + "resolved": "https://registry.npmmirror.com/remark-stringify/-/remark-stringify-9.0.1.tgz", + "integrity": "sha512-mWmNg3ZtESvZS8fv5PTvaPckdL4iNlCHTt8/e/8oN08nArHRHjNZMKzA/YW3+p7/lYqIw4nx1XsjCBo/AxNChg==", + "requires": { + "mdast-util-to-markdown": "^0.6.0" + } + }, + "remove-trailing-separator": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/remove-trailing-separator/-/remove-trailing-separator-1.1.0.tgz", + "integrity": "sha1-wkvOKig62tW8P1jg1IJJuSN52O8=" + }, + "renderkid": { + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/renderkid/-/renderkid-2.0.7.tgz", + "integrity": "sha512-oCcFyxaMrKsKcTY59qnCAtmDVSLfPbrv6A3tVbPdFMMrv5jaK10V6m40cKsoPNhAqN6rmHW9sswW4o3ruSrwUQ==", + "requires": { + "css-select": "^4.1.3", + "dom-converter": "^0.2.0", + "htmlparser2": "^6.1.0", + "lodash": "^4.17.21", + "strip-ansi": "^3.0.1" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=" + }, + "strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", + "requires": { + "ansi-regex": "^2.0.0" + } + } + } + }, + "repeat-string": { + "version": "1.6.1", + "resolved": "https://registry.npmmirror.com/repeat-string/-/repeat-string-1.6.1.tgz", + "integrity": "sha512-PV0dzCYDNfRi1jCDbJzpW7jNNDRuCOG/jI5ctQcGKt/clZD+YcPS3yIlWuTJMmESC8aevCFmWJy5wjAFgNqN6w==" + }, + "request": { + "version": "2.88.2", + "resolved": "https://registry.npmmirror.com/request/-/request-2.88.2.tgz", + "integrity": "sha512-MsvtOrfG9ZcrOwAW+Qi+F6HbD0CWXEh9ou77uOb7FM2WPhwT7smM833PzanhJLsgXjN89Ir6V2PczXNnMpwKhw==", + "requires": { + "aws-sign2": "~0.7.0", + "aws4": "^1.8.0", + "caseless": "~0.12.0", + "combined-stream": "~1.0.6", + "extend": "~3.0.2", + "forever-agent": "~0.6.1", + "form-data": "~2.3.2", + "har-validator": "~5.1.3", + "http-signature": "~1.2.0", + "is-typedarray": "~1.0.0", + "isstream": "~0.1.2", + "json-stringify-safe": "~5.0.1", + "mime-types": "~2.1.19", + "oauth-sign": "~0.9.0", + "performance-now": "^2.1.0", + "qs": "~6.5.2", + "safe-buffer": "^5.1.2", + "tough-cookie": "~2.5.0", + "tunnel-agent": "^0.6.0", + "uuid": "^3.3.2" + }, + "dependencies": { + "form-data": { + "version": "2.3.3", + "resolved": "https://registry.npmmirror.com/form-data/-/form-data-2.3.3.tgz", + "integrity": "sha512-1lLKB2Mu3aGP1Q/2eCOx0fNbRMe7XdwktwOruhfqqd0rIJWwN4Dh+E3hrPSlDCXnSR7UtZ1N38rVXm+6+MEhJQ==", + "requires": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.6", + "mime-types": "^2.1.12" + } + }, + "qs": { + "version": "6.5.3", + "resolved": "https://registry.npmmirror.com/qs/-/qs-6.5.3.tgz", + "integrity": "sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA==" + }, + "uuid": { + "version": "3.4.0", + "resolved": "https://registry.npmmirror.com/uuid/-/uuid-3.4.0.tgz", + "integrity": "sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A==" + } + } + }, + "require-directory": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", + "integrity": "sha1-jGStX9MNqxyXbiNE/+f3kqam30I=" + }, + "require-from-string": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/require-from-string/-/require-from-string-2.0.2.tgz", + "integrity": "sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw==" + }, + "require-main-filename": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-2.0.0.tgz", + "integrity": "sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg==" + }, + "require-package-name": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/require-package-name/-/require-package-name-2.0.1.tgz", + "integrity": "sha1-wR6XJ2tluOKSP3Xav1+y7ww4Qbk=" + }, + "requires-port": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/requires-port/-/requires-port-1.0.0.tgz", + "integrity": "sha1-kl0mAdOaxIXgkc8NpcbmlNw9yv8=" + }, + "resolve": { + "version": "1.22.0", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.22.0.tgz", + "integrity": "sha512-Hhtrw0nLeSrFQ7phPp4OOcVjLPIeMnRlr5mcnVuMe7M/7eBn98A3hmFRLoFo3DLZkivSYwhRUJTyPyWAk56WLw==", + "requires": { + "is-core-module": "^2.8.1", + "path-parse": "^1.0.7", + "supports-preserve-symlinks-flag": "^1.0.0" + } + }, + "resolve-alpn": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/resolve-alpn/-/resolve-alpn-1.2.1.tgz", + "integrity": "sha512-0a1F4l73/ZFZOakJnQ3FvkJ2+gSTQWz/r2KE5OdDY0TxPm5h4GkqkWWfM47T7HsbnOtcJVEF4epCVy6u7Q3K+g==" + }, + "resolve-cwd": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/resolve-cwd/-/resolve-cwd-3.0.0.tgz", + "integrity": "sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg==", + "requires": { + "resolve-from": "^5.0.0" + } + }, + "resolve-from": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-5.0.0.tgz", + "integrity": "sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw==" + }, + "resolve-url": { + "version": "0.2.1", + "resolved": "https://registry.npmmirror.com/resolve-url/-/resolve-url-0.2.1.tgz", + "integrity": "sha512-ZuF55hVUQaaczgOIwqWzkEcEidmlD/xl44x1UZnhOXcYuFN2S6+rcxpG+C1N3So0wvNI3DmJICUFfu2SxhBmvg==" + }, + "resolve-url-loader": { + "version": "3.1.4", + "resolved": "https://registry.npmmirror.com/resolve-url-loader/-/resolve-url-loader-3.1.4.tgz", + "integrity": "sha512-D3sQ04o0eeQEySLrcz4DsX3saHfsr8/N6tfhblxgZKXxMT2Louargg12oGNfoTRLV09GXhVUe5/qgA5vdgNigg==", + "requires": { + "adjust-sourcemap-loader": "3.0.0", + "camelcase": "5.3.1", + "compose-function": "3.0.3", + "convert-source-map": "1.7.0", + "es6-iterator": "2.0.3", + "loader-utils": "1.2.3", + "postcss": "7.0.36", + "rework": "1.0.1", + "rework-visit": "1.0.0", + "source-map": "0.6.1" + }, + "dependencies": { + "chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmmirror.com/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "convert-source-map": { + "version": "1.7.0", + "resolved": "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-1.7.0.tgz", + "integrity": "sha512-4FJkXzKXEDB1snCFZlLP4gpC3JILicCpGbzG9f9G7tGqGCzETQ2hWPrcinA9oU4wtf2biUaEH5065UnMeR33oA==", + "requires": { + "safe-buffer": "~5.1.1" + } + }, + "emojis-list": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/emojis-list/-/emojis-list-2.1.0.tgz", + "integrity": "sha512-knHEZMgs8BB+MInokmNTg/OyPlAddghe1YBgNwJBc5zsJi/uyIcXoSDsL/W9ymOsBoBGdPIHXYJ9+qKFwRwDng==" + }, + "json5": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "requires": { + "minimist": "^1.2.0" + } + }, + "loader-utils": { + "version": "1.2.3", + "resolved": "https://registry.npmmirror.com/loader-utils/-/loader-utils-1.2.3.tgz", + "integrity": "sha512-fkpz8ejdnEMG3s37wGL07iSBDg99O9D5yflE9RGNH3hRdx9SOwYfnGYdZOUIZitN8E+E2vkq3MUMYMvPYl5ZZA==", + "requires": { + "big.js": "^5.2.2", + "emojis-list": "^2.0.0", + "json5": "^1.0.1" + } + }, + "postcss": { + "version": "7.0.36", + "resolved": "https://registry.npmmirror.com/postcss/-/postcss-7.0.36.tgz", + "integrity": "sha512-BebJSIUMwJHRH0HAQoxN4u1CN86glsrwsW0q7T+/m44eXOUAxSNdHRkNZPYz5vVUbg17hFgOQDE7fZk7li3pZw==", + "requires": { + "chalk": "^2.4.2", + "source-map": "^0.6.1", + "supports-color": "^6.1.0" + }, + "dependencies": { + "supports-color": { + "version": "6.1.0", + "resolved": "https://registry.npmmirror.com/supports-color/-/supports-color-6.1.0.tgz", + "integrity": "sha512-qe1jfm1Mg7Nq/NSh6XE24gPXROEVsWHxC1LIx//XNlD9iw7YZQGjZNjYN7xGaEG6iKdA8EtNFW6R0gjnVXp+wQ==", + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "responselike": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/responselike/-/responselike-2.0.0.tgz", + "integrity": "sha512-xH48u3FTB9VsZw7R+vvgaKeLKzT6jOogbQhEe/jewwnZgzPcnyWui2Av6JpoYZF/91uueC+lqhWqeURw5/qhCw==", + "requires": { + "lowercase-keys": "^2.0.0" + } + }, + "restore-cursor": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/restore-cursor/-/restore-cursor-3.1.0.tgz", + "integrity": "sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA==", + "requires": { + "onetime": "^5.1.0", + "signal-exit": "^3.0.2" + } + }, + "retext-english": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/retext-english/-/retext-english-3.0.4.tgz", + "integrity": "sha512-yr1PgaBDde+25aJXrnt3p1jvT8FVLVat2Bx8XeAWX13KXo8OT+3nWGU3HWxM4YFJvmfqvJYJZG2d7xxaO774gw==", + "requires": { + "parse-english": "^4.0.0", + "unherit": "^1.0.4" + } + }, + "retry": { + "version": "0.12.0", + "resolved": "https://registry.npmjs.org/retry/-/retry-0.12.0.tgz", + "integrity": "sha1-G0KmJmoh8HQh0bC1S33BZ7AcATs=" + }, + "reusify": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/reusify/-/reusify-1.0.4.tgz", + "integrity": "sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw==" + }, + "rework": { + "version": "1.0.1", + "resolved": "https://registry.npmmirror.com/rework/-/rework-1.0.1.tgz", + "integrity": "sha512-eEjL8FdkdsxApd0yWVZgBGzfCQiT8yqSc2H1p4jpZpQdtz7ohETiDMoje5PlM8I9WgkqkreVxFUKYOiJdVWDXw==", + "requires": { + "convert-source-map": "^0.3.3", + "css": "^2.0.0" + }, + "dependencies": { + "convert-source-map": { + "version": "0.3.5", + "resolved": "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-0.3.5.tgz", + "integrity": "sha512-+4nRk0k3oEpwUB7/CalD7xE2z4VmtEnnq0GO2IPTkrooTrAhEsWvuLF5iWP1dXrwluki/azwXV1ve7gtYuPldg==" + } + } + }, + "rework-visit": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/rework-visit/-/rework-visit-1.0.0.tgz", + "integrity": "sha512-W6V2fix7nCLUYX1v6eGPrBOZlc03/faqzP4sUxMAJMBMOPYhfV/RyLegTufn5gJKaOITyi+gvf0LXDZ9NzkHnQ==" + }, + "rimraf": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz", + "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==", + "requires": { + "glob": "^7.1.3" + } + }, + "run-async": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/run-async/-/run-async-2.4.1.tgz", + "integrity": "sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ==" + }, + "run-parallel": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/run-parallel/-/run-parallel-1.2.0.tgz", + "integrity": "sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA==", + "requires": { + "queue-microtask": "^1.2.2" + } + }, + "rxjs": { + "version": "6.6.7", + "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-6.6.7.tgz", + "integrity": "sha512-hTdwr+7yYNIT5n4AMYp85KA6yw2Va0FLa3Rguvbpa4W3I5xynaBZo41cM3XM+4Q6fRMj3sBYIR1VAmZMXYJvRQ==", + "requires": { + "tslib": "^1.9.0" + }, + "dependencies": { + "tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + } + } + }, + "safe-buffer": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.1.2.tgz", + "integrity": "sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g==" + }, + "safer-buffer": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", + "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==" + }, + "sanitize-html": { + "version": "1.27.5", + "resolved": "https://registry.npmmirror.com/sanitize-html/-/sanitize-html-1.27.5.tgz", + "integrity": "sha512-M4M5iXDAUEcZKLXkmk90zSYWEtk5NH3JmojQxKxV371fnMh+x9t1rqdmXaGoyEHw3z/X/8vnFhKjGL5xFGOJ3A==", + "requires": { + "htmlparser2": "^4.1.0", + "lodash": "^4.17.15", + "parse-srcset": "^1.0.2", + "postcss": "^7.0.27" + }, + "dependencies": { + "domhandler": { + "version": "3.3.0", + "resolved": "https://registry.npmmirror.com/domhandler/-/domhandler-3.3.0.tgz", + "integrity": "sha512-J1C5rIANUbuYK+FuFL98650rihynUOEzRLxW+90bKZRWB6A1X1Tf82GxR1qAWLyfNPRvjqfip3Q5tdYlmAa9lA==", + "requires": { + "domelementtype": "^2.0.1" + } + }, + "htmlparser2": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/htmlparser2/-/htmlparser2-4.1.0.tgz", + "integrity": "sha512-4zDq1a1zhE4gQso/c5LP1OtrhYTncXNSpvJYtWJBtXAETPlMfi3IFNjGuQbYLuVY4ZR0QMqRVvo4Pdy9KLyP8Q==", + "requires": { + "domelementtype": "^2.0.1", + "domhandler": "^3.0.0", + "domutils": "^2.0.0", + "entities": "^2.0.0" + } + }, + "picocolors": { + "version": "0.2.1", + "resolved": "https://registry.npmmirror.com/picocolors/-/picocolors-0.2.1.tgz", + "integrity": "sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA==" + }, + "postcss": { + "version": "7.0.39", + "resolved": "https://registry.npmmirror.com/postcss/-/postcss-7.0.39.tgz", + "integrity": "sha512-yioayjNbHn6z1/Bywyb2Y4s3yvDAeXGOyxqD+LnVOinq6Mdmd++SW2wUNVzavyyHxd6+DxzWGIuosg6P1Rj8uA==", + "requires": { + "picocolors": "^0.2.1", + "source-map": "^0.6.1" + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "sass": { + "version": "1.51.0", + "resolved": "https://registry.npmmirror.com/sass/-/sass-1.51.0.tgz", + "integrity": "sha512-haGdpTgywJTvHC2b91GSq+clTKGbtkkZmVAb82jZQN/wTy6qs8DdFm2lhEQbEwrY0QDRgSQ3xDurqM977C3noA==", + "peer": true, + "requires": { + "chokidar": ">=3.0.0 <4.0.0", + "immutable": "^4.0.0", + "source-map-js": ">=0.6.2 <2.0.0" + }, + "dependencies": { + "immutable": { + "version": "4.0.0", + "resolved": "https://registry.npmmirror.com/immutable/-/immutable-4.0.0.tgz", + "integrity": "sha512-zIE9hX70qew5qTUjSS7wi1iwj/l7+m54KWU247nhM3v806UdGj1yDndXj+IOYxxtW9zyLI+xqFNZjTuDaLUqFw==", + "peer": true + } + } + }, + "sass-graph": { + "version": "2.2.5", + "resolved": "https://registry.npmmirror.com/sass-graph/-/sass-graph-2.2.5.tgz", + "integrity": "sha512-VFWDAHOe6mRuT4mZRd4eKE+d8Uedrk6Xnh7Sh9b4NGufQLQjOrvf/MQoOdx+0s92L89FeyUUNfU597j/3uNpag==", + "requires": { + "glob": "^7.0.0", + "lodash": "^4.0.0", + "scss-tokenizer": "^0.2.3", + "yargs": "^13.3.2" + }, + "dependencies": { + "ansi-regex": { + "version": "4.1.1", + "resolved": "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-4.1.1.tgz", + "integrity": "sha512-ILlv4k/3f6vfQ4OoP2AGvirOktlQ98ZEL1k9FaQjxa3L1abBgbuTDAdPOpvbGncC0BTVQrl+OM8xZGK6tWXt7g==" + }, + "cliui": { + "version": "5.0.0", + "resolved": "https://registry.npmmirror.com/cliui/-/cliui-5.0.0.tgz", + "integrity": "sha512-PYeGSEmmHM6zvoef2w8TPzlrnNpXIjTipYK780YswmIP9vjxmd6Y2a3CB2Ks6/AU8NHjZugXvo8w3oWM2qnwXA==", + "requires": { + "string-width": "^3.1.0", + "strip-ansi": "^5.2.0", + "wrap-ansi": "^5.1.0" + } + }, + "emoji-regex": { + "version": "7.0.3", + "resolved": "https://registry.npmmirror.com/emoji-regex/-/emoji-regex-7.0.3.tgz", + "integrity": "sha512-CwBLREIQ7LvYFB0WyRvwhq5N5qPhc6PMjD6bYggFlI5YyDgl+0vxq5VHbMOFqLg7hfWzmu8T5Z1QofhmTIhItA==" + }, + "find-up": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/find-up/-/find-up-3.0.0.tgz", + "integrity": "sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg==", + "requires": { + "locate-path": "^3.0.0" + } + }, + "is-fullwidth-code-point": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz", + "integrity": "sha512-VHskAKYM8RfSFXwee5t5cbN5PZeq1Wrh6qd5bkyiXIf6UQcN6w/A0eXM9r6t8d+GYOh+o6ZhiEnb88LN/Y8m2w==" + }, + "locate-path": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/locate-path/-/locate-path-3.0.0.tgz", + "integrity": "sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A==", + "requires": { + "p-locate": "^3.0.0", + "path-exists": "^3.0.0" + } + }, + "p-locate": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/p-locate/-/p-locate-3.0.0.tgz", + "integrity": "sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ==", + "requires": { + "p-limit": "^2.0.0" + } + }, + "path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha512-bpC7GYwiDYQ4wYLe+FA8lhRjhQCMcQGuSgGGqDkg/QerRWw9CmGRT0iSOVRSZJ29NMLZgIzqaljJ63oaL4NIJQ==" + }, + "string-width": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/string-width/-/string-width-3.1.0.tgz", + "integrity": "sha512-vafcv6KjVZKSgz06oM/H6GDBrAtz8vdhQakGjFIvNrHA6y3HCF1CInLy+QLq8dTJPQ1b+KDUqDFctkdRW44e1w==", + "requires": { + "emoji-regex": "^7.0.1", + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^5.1.0" + } + }, + "strip-ansi": { + "version": "5.2.0", + "resolved": "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-5.2.0.tgz", + "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", + "requires": { + "ansi-regex": "^4.1.0" + } + }, + "wrap-ansi": { + "version": "5.1.0", + "resolved": "https://registry.npmmirror.com/wrap-ansi/-/wrap-ansi-5.1.0.tgz", + "integrity": "sha512-QC1/iN/2/RPVJ5jYK8BGttj5z83LmSKmvbvrXPNCLZSEb32KKVDJDl/MOt2N01qU2H/FkzEa9PKto1BqDjtd7Q==", + "requires": { + "ansi-styles": "^3.2.0", + "string-width": "^3.0.0", + "strip-ansi": "^5.0.0" + } + }, + "yargs": { + "version": "13.3.2", + "resolved": "https://registry.npmmirror.com/yargs/-/yargs-13.3.2.tgz", + "integrity": "sha512-AX3Zw5iPruN5ie6xGRIDgqkT+ZhnRlZMLMHAs8tg7nRruy2Nb+i5o9bwghAogtM08q1dpr2LVoS8KSTMYpWXUw==", + "requires": { + "cliui": "^5.0.0", + "find-up": "^3.0.0", + "get-caller-file": "^2.0.1", + "require-directory": "^2.1.1", + "require-main-filename": "^2.0.0", + "set-blocking": "^2.0.0", + "string-width": "^3.0.0", + "which-module": "^2.0.0", + "y18n": "^4.0.0", + "yargs-parser": "^13.1.2" + } + }, + "yargs-parser": { + "version": "13.1.2", + "resolved": "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-13.1.2.tgz", + "integrity": "sha512-3lbsNRf/j+A4QuSZfDRA7HRSfWrzO0YjqTJd5kjAq37Zep1CEgaYmrH9Q3GwPiB9cHyd1Y1UwggGhJGoxipbzg==", + "requires": { + "camelcase": "^5.0.0", + "decamelize": "^1.2.0" + } + } + } + }, + "sass-loader": { + "version": "10.2.1", + "resolved": "https://registry.npmmirror.com/sass-loader/-/sass-loader-10.2.1.tgz", + "integrity": "sha512-RRvWl+3K2LSMezIsd008ErK4rk6CulIMSwrcc2aZvjymUgKo/vjXGp1rSWmfTUX7bblEOz8tst4wBwWtCGBqKA==", + "requires": { + "klona": "^2.0.4", + "loader-utils": "^2.0.0", + "neo-async": "^2.6.2", + "schema-utils": "^3.0.0", + "semver": "^7.3.2" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "sax": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/sax/-/sax-1.2.4.tgz", + "integrity": "sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw==" + }, + "scheduler": { + "version": "0.20.1", + "integrity": "sha512-LKTe+2xNJBNxu/QhHvDR14wUXHRQbVY5ZOYpOGWRzhydZUqrLb2JBvLPY7cAqFmqrWuDED0Mjk7013SZiOz6Bw==", + "requires": { + "loose-envify": "^1.1.0", + "object-assign": "^4.1.1" + } + }, + "schema-utils": { + "version": "2.7.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-2.7.1.tgz", + "integrity": "sha512-SHiNtMOUGWBQJwzISiVYKu82GiV4QYGePp3odlY1tuKO7gPtphAT5R/py0fA6xtbgLL/RvtJZnU9b8s0F1q0Xg==", + "requires": { + "@types/json-schema": "^7.0.5", + "ajv": "^6.12.4", + "ajv-keywords": "^3.5.2" + } + }, + "scss-tokenizer": { + "version": "0.2.3", + "resolved": "https://registry.npmmirror.com/scss-tokenizer/-/scss-tokenizer-0.2.3.tgz", + "integrity": "sha512-dYE8LhncfBUar6POCxMTm0Ln+erjeczqEvCJib5/7XNkdw1FkUGgwMPY360FY0FgPWQxHWCx29Jl3oejyGLM9Q==", + "requires": { + "js-base64": "^2.1.8", + "source-map": "^0.4.2" + }, + "dependencies": { + "source-map": { + "version": "0.4.4", + "resolved": "https://registry.npmmirror.com/source-map/-/source-map-0.4.4.tgz", + "integrity": "sha512-Y8nIfcb1s/7DcobUz1yOO1GSp7gyL+D9zLHDehT7iRESqGSxjJ448Sg7rvfgsRJCnKLdSl11uGf0s9X80cH0/A==", + "requires": { + "amdefine": ">=0.0.4" + } + } + } + }, + "section-matter": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/section-matter/-/section-matter-1.0.0.tgz", + "integrity": "sha512-vfD3pmTzGpufjScBh50YHKzEu2lxBWhVEHsNGoEXmCmn2hKGfeNLYMzCJpe8cD7gqX7TJluOVpBkAequ6dgMmA==", + "requires": { + "extend-shallow": "^2.0.1", + "kind-of": "^6.0.0" + } + }, + "semver": { + "version": "7.3.7", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.7.tgz", + "integrity": "sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g==", + "requires": { + "lru-cache": "^6.0.0" + } + }, + "semver-diff": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/semver-diff/-/semver-diff-3.1.1.tgz", + "integrity": "sha512-GX0Ix/CJcHyB8c4ykpHGIAvLyOwOobtM/8d+TQkAd81/bEjgPHrfba41Vpesr7jX/t8Uh+R3EX9eAS5be+jQYg==", + "requires": { + "semver": "^6.3.0" + }, + "dependencies": { + "semver": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.0.tgz", + "integrity": "sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==" + } + } + }, + "send": { + "version": "0.18.0", + "resolved": "https://registry.npmjs.org/send/-/send-0.18.0.tgz", + "integrity": "sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg==", + "requires": { + "debug": "2.6.9", + "depd": "2.0.0", + "destroy": "1.2.0", + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "etag": "~1.8.1", + "fresh": "0.5.2", + "http-errors": "2.0.0", + "mime": "1.6.0", + "ms": "2.1.3", + "on-finished": "2.4.1", + "range-parser": "~1.2.1", + "statuses": "2.0.1" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "mime": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/mime/-/mime-1.6.0.tgz", + "integrity": "sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg==" + }, + "ms": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==" + } + } + }, + "sentence-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/sentence-case/-/sentence-case-3.0.4.tgz", + "integrity": "sha512-8LS0JInaQMCRoQ7YUytAo/xUu5W2XnQxV2HI/6uM6U7CITS1RqPElr30V6uIqyMKM9lJGRVFy5/4CuzcixNYSg==", + "requires": { + "no-case": "^3.0.4", + "tslib": "^2.0.3", + "upper-case-first": "^2.0.2" + } + }, + "serialize-javascript": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-5.0.1.tgz", + "integrity": "sha512-SaaNal9imEO737H2c05Og0/8LUXG7EnsZyMa8MzkmuHoELfT6txuj0cMqRj6zfPKnmQ1yasR4PCJc8x+M4JSPA==", + "requires": { + "randombytes": "^2.1.0" + } + }, + "serve-static": { + "version": "1.15.0", + "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.15.0.tgz", + "integrity": "sha512-XGuRDNjXUijsUL0vl6nSD7cwURuzEgglbOaFuZM9g3kwDXOWVTck0jLzjPzGD+TazWbboZYu52/9/XPdUgne9g==", + "requires": { + "encodeurl": "~1.0.2", + "escape-html": "~1.0.3", + "parseurl": "~1.3.3", + "send": "0.18.0" + } + }, + "set-blocking": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/set-blocking/-/set-blocking-2.0.0.tgz", + "integrity": "sha1-BF+XgtARrppoA93TgrJDkrPYkPc=" + }, + "setimmediate": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/setimmediate/-/setimmediate-1.0.5.tgz", + "integrity": "sha1-KQy7Iy4waULX1+qbg3Mqt4VvgoU=" + }, + "setprototypeof": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/setprototypeof/-/setprototypeof-1.2.0.tgz", + "integrity": "sha512-E5LDX7Wrp85Kil5bhZv46j8jOeboKq5JMmYM3gVGdGH8xFpPWXUMsNrlODCrkoxMEeNi/XZIwuRvY4XNwYMJpw==" + }, + "shallow-clone": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/shallow-clone/-/shallow-clone-3.0.1.tgz", + "integrity": "sha512-/6KqX+GVUdqPuPPd2LxDDxzX6CAbjJehAAOKlNpqqUpAqPM6HeL8f+o3a+JsyGjn2lv0WY8UsTgUJjU9Ok55NA==", + "requires": { + "kind-of": "^6.0.2" + } + }, + "shallow-compare": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/shallow-compare/-/shallow-compare-1.2.2.tgz", + "integrity": "sha512-LUMFi+RppPlrHzbqmFnINTrazo0lPNwhcgzuAXVVcfy/mqPDrQmHAyz5bvV0gDAuRFrk804V0HpQ6u9sZ0tBeg==" + }, + "sharp": { + "version": "0.30.4", + "resolved": "https://registry.npmjs.org/sharp/-/sharp-0.30.4.tgz", + "integrity": "sha512-3Onig53Y6lji4NIZo69s14mERXXY/GV++6CzOYx/Rd8bnTwbhFbL09WZd7Ag/CCnA0WxFID8tkY0QReyfL6v0Q==", + "requires": { + "color": "^4.2.3", + "detect-libc": "^2.0.1", + "node-addon-api": "^4.3.0", + "prebuild-install": "^7.0.1", + "semver": "^7.3.7", + "simple-get": "^4.0.1", + "tar-fs": "^2.1.1", + "tunnel-agent": "^0.6.0" + }, + "dependencies": { + "detect-libc": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.1.tgz", + "integrity": "sha512-463v3ZeIrcWtdgIg6vI6XUncguvr2TnGl4SzDXinkt9mSLpBJKXT3mW6xT3VQdDN11+WVs29pgvivTc4Lp8v+w==" + }, + "node-addon-api": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/node-addon-api/-/node-addon-api-4.3.0.tgz", + "integrity": "sha512-73sE9+3UaLYYFmDsFZnqCInzPyh3MqIwZO9cw58yIqAZhONrrabrYyYe3TuIqtIiOuTXVhsGau8hcrhhwSsDIQ==" + } + } + }, + "shebang-command": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-1.2.0.tgz", + "integrity": "sha1-RKrGW2lbAzmJaMOfNj/uXer98eo=", + "requires": { + "shebang-regex": "^1.0.0" + } + }, + "shebang-regex": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-1.0.0.tgz", + "integrity": "sha1-2kL0l0DAtC2yypcoVxyxkMmO/qM=" + }, + "shell-quote": { + "version": "1.7.3", + "resolved": "https://registry.npmjs.org/shell-quote/-/shell-quote-1.7.3.tgz", + "integrity": "sha512-Vpfqwm4EnqGdlsBFNmHhxhElJYrdfcxPThu+ryKS5J8L/fhAwLazFZtq+S+TWZ9ANj2piSQLGj6NQg+lKPmxrw==" + }, + "side-channel": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/side-channel/-/side-channel-1.0.4.tgz", + "integrity": "sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw==", + "requires": { + "call-bind": "^1.0.0", + "get-intrinsic": "^1.0.2", + "object-inspect": "^1.9.0" + } + }, + "signal-exit": { + "version": "3.0.7", + "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.7.tgz", + "integrity": "sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==" + }, + "signedsource": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/signedsource/-/signedsource-1.0.0.tgz", + "integrity": "sha1-HdrOSYF5j5O9gzlzgD2A1S6TrWo=" + }, + "simple-concat": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/simple-concat/-/simple-concat-1.0.1.tgz", + "integrity": "sha512-cSFtAPtRhljv69IK0hTVZQ+OfE9nePi/rtJmw5UjHeVyVroEqJXP1sFztKUy1qU+xvz3u/sfYJLa947b7nAN2Q==" + }, + "simple-get": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/simple-get/-/simple-get-4.0.1.tgz", + "integrity": "sha512-brv7p5WgH0jmQJr1ZDDfKDOSeWWg+OVypG99A/5vYGPqJ6pxiaHLy8nxtFjBA7oMa01ebA9gfh1uMCFqOuXxvA==", + "requires": { + "decompress-response": "^6.0.0", + "once": "^1.3.1", + "simple-concat": "^1.0.0" + } + }, + "simple-swizzle": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/simple-swizzle/-/simple-swizzle-0.2.2.tgz", + "integrity": "sha1-pNprY1/8zMoz9w0Xy5JZLeleVXo=", + "requires": { + "is-arrayish": "^0.3.1" + }, + "dependencies": { + "is-arrayish": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.3.2.tgz", + "integrity": "sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ==" + } + } + }, + "sisteransi": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", + "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==" + }, + "slash": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", + "integrity": "sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q==" + }, + "slice-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/slice-ansi/-/slice-ansi-4.0.0.tgz", + "integrity": "sha512-qMCMfhY040cVHT43K9BFygqYbUPFZKHOg7K73mtTWJRb8pyP3fzf4Ixd5SzdEJQ6MRUg/WBnOLxghZtKKurENQ==", + "requires": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + }, + "dependencies": { + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "requires": { + "color-convert": "^2.0.1" + } + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + } + } + }, + "slugify": { + "version": "1.6.5", + "resolved": "https://registry.npmjs.org/slugify/-/slugify-1.6.5.tgz", + "integrity": "sha512-8mo9bslnBO3tr5PEVFzMPIWwWnipGS0xVbYf65zxDqfNwmzYn1LpiKNrR6DlClusuvo+hDHd1zKpmfAe83NQSQ==" + }, + "snake-case": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/snake-case/-/snake-case-3.0.4.tgz", + "integrity": "sha512-LAOh4z89bGQvl9pFfNF8V146i7o7/CqFPbqzYgP+yYzDIDeS9HaNFtXABamRW+AQzEVODcvE79ljJ+8a9YSdMg==", + "requires": { + "dot-case": "^3.0.4", + "tslib": "^2.0.3" + } + }, + "socket.io": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/socket.io/-/socket.io-3.1.2.tgz", + "integrity": "sha512-JubKZnTQ4Z8G4IZWtaAZSiRP3I/inpy8c/Bsx2jrwGrTbKeVU5xd6qkKMHpChYeM3dWZSO0QACiGK+obhBNwYw==", + "requires": { + "@types/cookie": "^0.4.0", + "@types/cors": "^2.8.8", + "@types/node": ">=10.0.0", + "accepts": "~1.3.4", + "base64id": "~2.0.0", + "debug": "~4.3.1", + "engine.io": "~4.1.0", + "socket.io-adapter": "~2.1.0", + "socket.io-parser": "~4.0.3" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "socket.io-adapter": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/socket.io-adapter/-/socket.io-adapter-2.1.0.tgz", + "integrity": "sha512-+vDov/aTsLjViYTwS9fPy5pEtTkrbEKsw2M+oVSoFGw6OD1IpvlV1VPhUzNbofCQ8oyMbdYJqDtGdmHQK6TdPg==" + }, + "socket.io-client": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/socket.io-client/-/socket.io-client-3.1.3.tgz", + "integrity": "sha512-4sIGOGOmCg3AOgGi7EEr6ZkTZRkrXwub70bBB/F0JSkMOUFpA77WsL87o34DffQQ31PkbMUIadGOk+3tx1KGbw==", + "requires": { + "@types/component-emitter": "^1.2.10", + "backo2": "~1.0.2", + "component-emitter": "~1.3.0", + "debug": "~4.3.1", + "engine.io-client": "~4.1.0", + "parseuri": "0.0.6", + "socket.io-parser": "~4.0.4" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "socket.io-parser": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/socket.io-parser/-/socket.io-parser-4.0.4.tgz", + "integrity": "sha512-t+b0SS+IxG7Rxzda2EVvyBZbvFPBCjJoyHuE0P//7OAsN23GItzDRdWa6ALxZI/8R5ygK7jAR6t028/z+7295g==", + "requires": { + "@types/component-emitter": "^1.2.10", + "component-emitter": "~1.3.0", + "debug": "~4.3.1" + }, + "dependencies": { + "debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "requires": { + "ms": "2.1.2" + } + } + } + }, + "source-list-map": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/source-list-map/-/source-list-map-2.0.1.tgz", + "integrity": "sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw==" + }, + "source-map": { + "version": "0.7.3", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.7.3.tgz", + "integrity": "sha512-CkCj6giN3S+n9qrYiBTX5gystlENnRW5jZeNLHpe6aue+SrHcG5VYwujhW9s4dY31mEGsxBDrHR6oI69fTXsaQ==" + }, + "source-map-js": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/source-map-js/-/source-map-js-1.0.2.tgz", + "integrity": "sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw==" + }, + "source-map-resolve": { + "version": "0.5.3", + "resolved": "https://registry.npmmirror.com/source-map-resolve/-/source-map-resolve-0.5.3.tgz", + "integrity": "sha512-Htz+RnsXWk5+P2slx5Jh3Q66vhQj1Cllm0zvnaY98+NFx+Dv2CF/f5O/t8x+KaNdrdIAsruNzoh/KpialbqAnw==", + "requires": { + "atob": "^2.1.2", + "decode-uri-component": "^0.2.0", + "resolve-url": "^0.2.1", + "source-map-url": "^0.4.0", + "urix": "^0.1.0" + } + }, + "source-map-support": { + "version": "0.5.21", + "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.5.21.tgz", + "integrity": "sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w==", + "requires": { + "buffer-from": "^1.0.0", + "source-map": "^0.6.0" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "source-map-url": { + "version": "0.4.1", + "resolved": "https://registry.npmmirror.com/source-map-url/-/source-map-url-0.4.1.tgz", + "integrity": "sha512-cPiFOTLUKvJFIg4SKVScy4ilPPW6rFgMgfuZJPNoDuMs3nC1HbMUycBoJw77xFIp6z1UJQJOfx6C9GMH80DiTw==" + }, + "space-separated-tokens": { + "version": "1.1.5", + "resolved": "https://registry.npmmirror.com/space-separated-tokens/-/space-separated-tokens-1.1.5.tgz", + "integrity": "sha512-q/JSVd1Lptzhf5bkYm4ob4iWPjx0KiRe3sRFBNrVqbJkFaBm5vbbowy1mymoPNLRa52+oadOhJ+K49wsSeSjTA==" + }, + "spdx-correct": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/spdx-correct/-/spdx-correct-3.1.1.tgz", + "integrity": "sha512-cOYcUWwhCuHCXi49RhFRCyJEK3iPj1Ziz9DpViV3tbZOwXD49QzIN3MpOLJNxh2qwq2lJJZaKMVw9qNi4jTC0w==", + "requires": { + "spdx-expression-parse": "^3.0.0", + "spdx-license-ids": "^3.0.0" + } + }, + "spdx-exceptions": { + "version": "2.3.0", + "resolved": "https://registry.npmmirror.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz", + "integrity": "sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A==" + }, + "spdx-expression-parse": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz", + "integrity": "sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q==", + "requires": { + "spdx-exceptions": "^2.1.0", + "spdx-license-ids": "^3.0.0" + } + }, + "spdx-license-ids": { + "version": "3.0.11", + "resolved": "https://registry.npmmirror.com/spdx-license-ids/-/spdx-license-ids-3.0.11.tgz", + "integrity": "sha512-Ctl2BrFiM0X3MANYgj3CkygxhRmr9mi6xhejbdO960nF6EDJApTYpn0BQnDKlnNBULKiCN1n3w9EBkHK8ZWg+g==" + }, + "split-on-first": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/split-on-first/-/split-on-first-1.1.0.tgz", + "integrity": "sha512-43ZssAJaMusuKWL8sKUBQXHWOpq8d6CfN/u1p4gUzfJkM05C8rxTmYrkIPTXapZpORA6LkkzcUulJ8FqA7Uudw==" + }, + "sponge-case": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/sponge-case/-/sponge-case-1.0.1.tgz", + "integrity": "sha512-dblb9Et4DAtiZ5YSUZHLl4XhH4uK80GhAZrVXdN4O2P4gQ40Wa5UIOPUHlA/nFd2PLblBZWUioLMMAVrgpoYcA==", + "requires": { + "tslib": "^2.0.3" + } + }, + "sprintf-js": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", + "integrity": "sha1-BOaSb2YolTVPPdAVIDYzuFcpfiw=" + }, + "sshpk": { + "version": "1.17.0", + "resolved": "https://registry.npmmirror.com/sshpk/-/sshpk-1.17.0.tgz", + "integrity": "sha512-/9HIEs1ZXGhSPE8X6Ccm7Nam1z8KcoCqPdI7ecm1N33EzAetWahvQWVqLZtaZQ+IDKX4IyA2o0gBzqIMkAagHQ==", + "requires": { + "asn1": "~0.2.3", + "assert-plus": "^1.0.0", + "bcrypt-pbkdf": "^1.0.0", + "dashdash": "^1.12.0", + "ecc-jsbn": "~0.1.1", + "getpass": "^0.1.1", + "jsbn": "~0.1.0", + "safer-buffer": "^2.0.2", + "tweetnacl": "~0.14.0" + } + }, + "st": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/st/-/st-2.0.0.tgz", + "integrity": "sha512-drN+aGYnrZPNYIymmNwIY7LXYJ8MqsqXj4fMRue3FOgGMdGjSX10fhJ3qx0sVQPhcWxhEaN4U/eWM4O4dbYNAw==", + "requires": { + "async-cache": "^1.1.0", + "bl": "^4.0.0", + "fd": "~0.0.2", + "graceful-fs": "^4.2.3", + "mime": "^2.4.4", + "negotiator": "~0.6.2" + } + }, + "stable": { + "version": "0.1.8", + "resolved": "https://registry.npmjs.org/stable/-/stable-0.1.8.tgz", + "integrity": "sha512-ji9qxRnOVfcuLDySj9qzhGSEFVobyt1kIOSkj1qZzYLzq7Tos/oUUWvotUPQLlrsidqsK6tBH89Bc9kL5zHA6w==" + }, + "stack-trace": { + "version": "0.0.10", + "resolved": "https://registry.npmjs.org/stack-trace/-/stack-trace-0.0.10.tgz", + "integrity": "sha1-VHxws0fo0ytOEI6hoqFZ5f3eGcA=" + }, + "stackframe": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/stackframe/-/stackframe-1.2.1.tgz", + "integrity": "sha512-h88QkzREN/hy8eRdyNhhsO7RSJ5oyTqxxmmn0dzBIMUclZsjpfmrsg81vp8mjjAs2vAZ72nyWxRUwSwmh0e4xg==" + }, + "statuses": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/statuses/-/statuses-2.0.1.tgz", + "integrity": "sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ==" + }, + "stdout-stream": { + "version": "1.4.1", + "resolved": "https://registry.npmmirror.com/stdout-stream/-/stdout-stream-1.4.1.tgz", + "integrity": "sha512-j4emi03KXqJWcIeF8eIXkjMFN1Cmb8gUlDYGeBALLPo5qdyTfA9bOtl8m33lRoC+vFMkP3gl0WsDr6+gzxbbTA==", + "requires": { + "readable-stream": "^2.0.1" + }, + "dependencies": { + "readable-stream": { + "version": "2.3.7", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-2.3.7.tgz", + "integrity": "sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "requires": { + "safe-buffer": "~5.1.0" + } + } + } + }, + "stream-parser": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/stream-parser/-/stream-parser-0.3.1.tgz", + "integrity": "sha1-FhhUhpRCACGhGC/wrxkRwSl2F3M=", + "requires": { + "debug": "2" + }, + "dependencies": { + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "requires": { + "ms": "2.0.0" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=" + } + } + }, + "streamsearch": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/streamsearch/-/streamsearch-0.1.2.tgz", + "integrity": "sha1-gIudDlb8Jz2Am6VzOOkpkZoanxo=" + }, + "strict-uri-encode": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/strict-uri-encode/-/strict-uri-encode-2.0.0.tgz", + "integrity": "sha1-ucczDHBChi9rFC3CdLvMWGbONUY=" + }, + "string_decoder": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz", + "integrity": "sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==", + "requires": { + "safe-buffer": "~5.2.0" + }, + "dependencies": { + "safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==" + } + } + }, + "string-env-interpolation": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/string-env-interpolation/-/string-env-interpolation-1.0.1.tgz", + "integrity": "sha512-78lwMoCcn0nNu8LszbP1UA7g55OeE4v7rCeWnM5B453rnNr4aq+5it3FEYtZrSEiMvHZOZ9Jlqb0OD0M2VInqg==" + }, + "string-natural-compare": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/string-natural-compare/-/string-natural-compare-3.0.1.tgz", + "integrity": "sha512-n3sPwynL1nwKi3WJ6AIsClwBMa0zTi54fn2oLU6ndfTSIO05xaznjSf15PcBZU6FNWbmN5Q6cxT4V5hGvB4taw==" + }, + "string-similarity": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/string-similarity/-/string-similarity-1.2.2.tgz", + "integrity": "sha512-IoHUjcw3Srl8nsPlW04U3qwWPk3oG2ffLM0tN853d/E/JlIvcmZmDY2Kz5HzKp4lEi2T7QD7Zuvjq/1rDw+XcQ==", + "requires": { + "lodash.every": "^4.6.0", + "lodash.flattendeep": "^4.4.0", + "lodash.foreach": "^4.5.0", + "lodash.map": "^4.6.0", + "lodash.maxby": "^4.6.0" + } + }, + "string-width": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "requires": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + } + }, + "string.prototype.matchall": { + "version": "4.0.7", + "resolved": "https://registry.npmjs.org/string.prototype.matchall/-/string.prototype.matchall-4.0.7.tgz", + "integrity": "sha512-f48okCX7JiwVi1NXCVWcFnZgADDC/n2vePlQ/KUCNqCikLLilQvwjMO8+BHVKvgzH0JB0J9LEPgxOGT02RoETg==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.3", + "es-abstract": "^1.19.1", + "get-intrinsic": "^1.1.1", + "has-symbols": "^1.0.3", + "internal-slot": "^1.0.3", + "regexp.prototype.flags": "^1.4.1", + "side-channel": "^1.0.4" + } + }, + "string.prototype.trimend": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/string.prototype.trimend/-/string.prototype.trimend-1.0.5.tgz", + "integrity": "sha512-I7RGvmjV4pJ7O3kdf+LXFpVfdNOxtCW/2C8f6jNiW4+PQchwxkCDzlk1/7p+Wl4bqFIZeF47qAHXLuHHWKAxog==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "string.prototype.trimstart": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/string.prototype.trimstart/-/string.prototype.trimstart-1.0.5.tgz", + "integrity": "sha512-THx16TJCGlsN0o6dl2o6ncWUsdgnLRSA23rRE5pyGBw/mLr3Ej/R2LaqCtgP8VNMGZsvMWnf9ooZPyY2bHvUFg==", + "requires": { + "call-bind": "^1.0.2", + "define-properties": "^1.1.4", + "es-abstract": "^1.19.5" + } + }, + "stringify-entities": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/stringify-entities/-/stringify-entities-3.1.0.tgz", + "integrity": "sha512-3FP+jGMmMV/ffZs86MoghGqAoqXAdxLrJP4GUdrDN1aIScYih5tuIO3eF4To5AJZ79KDZ8Fpdy7QJnK8SsL1Vg==", + "requires": { + "character-entities-html4": "^1.0.0", + "character-entities-legacy": "^1.0.0", + "xtend": "^4.0.0" + } + }, + "stringify-object": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/stringify-object/-/stringify-object-3.3.0.tgz", + "integrity": "sha512-rHqiFh1elqCQ9WPLIC8I0Q/g/wj5J1eMkyoiD6eoQApWHP0FtlK7rqnhmabL5VUY9JQCcqwwvlOaSuutekgyrw==", + "requires": { + "get-own-enumerable-property-symbols": "^3.0.0", + "is-obj": "^1.0.1", + "is-regexp": "^1.0.0" + }, + "dependencies": { + "is-obj": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-obj/-/is-obj-1.0.1.tgz", + "integrity": "sha1-PkcprB9f3gJc19g6iW2rn09n2w8=" + } + } + }, + "strip-ansi": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "requires": { + "ansi-regex": "^5.0.1" + } + }, + "strip-bom": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/strip-bom/-/strip-bom-3.0.0.tgz", + "integrity": "sha1-IzTBjpx1n3vdVv3vfprj1YjmjtM=" + }, + "strip-bom-string": { + "version": "1.0.0", + "resolved": "https://registry.npmmirror.com/strip-bom-string/-/strip-bom-string-1.0.0.tgz", + "integrity": "sha512-uCC2VHvQRYu+lMh4My/sFNmF2klFymLX1wHJeXnbEJERpV/ZsVuonzerjfrGpIGF7LBVa1O7i9kjiWvJiFck8g==" + }, + "strip-comments": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/strip-comments/-/strip-comments-1.0.2.tgz", + "integrity": "sha512-kL97alc47hoyIQSV165tTt9rG5dn4w1dNnBhOQ3bOU1Nc1hel09jnXANaHJ7vzHLd4Ju8kseDGzlev96pghLFw==", + "requires": { + "babel-extract-comments": "^1.0.0", + "babel-plugin-transform-object-rest-spread": "^6.26.0" + } + }, + "strip-eof": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/strip-eof/-/strip-eof-1.0.0.tgz", + "integrity": "sha1-u0P/VZim6wXYm1n80SnJgzE2Br8=" + }, + "strip-final-newline": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/strip-final-newline/-/strip-final-newline-2.0.0.tgz", + "integrity": "sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA==" + }, + "strip-indent": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/strip-indent/-/strip-indent-3.0.0.tgz", + "integrity": "sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ==", + "requires": { + "min-indent": "^1.0.0" + } + }, + "strip-json-comments": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", + "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==" + }, + "strip-outer": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/strip-outer/-/strip-outer-1.0.1.tgz", + "integrity": "sha512-k55yxKHwaXnpYGsOzg4Vl8+tDrWylxDEpknGjhTiZB8dFRU5rTo9CAzeycivxV3s+zlTKwrs6WxMxR95n26kwg==", + "requires": { + "escape-string-regexp": "^1.0.2" + } + }, + "strtok3": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/strtok3/-/strtok3-6.3.0.tgz", + "integrity": "sha512-fZtbhtvI9I48xDSywd/somNqgUHl2L2cstmXCCif0itOf96jeW18MBSyrLuNicYQVkvpOxkZtkzujiTJ9LW5Jw==", + "requires": { + "@tokenizer/token": "^0.3.0", + "peek-readable": "^4.1.0" + } + }, + "style-loader": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/style-loader/-/style-loader-2.0.0.tgz", + "integrity": "sha512-Z0gYUJmzZ6ZdRUqpg1r8GsaFKypE+3xAzuFeMuoHgjc9KZv3wMyCRjQIWEbhoFSq7+7yoHXySDJyyWQaPajeiQ==", + "requires": { + "loader-utils": "^2.0.0", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "style-to-js": { + "version": "1.1.0", + "resolved": "https://registry.npmmirror.com/style-to-js/-/style-to-js-1.1.0.tgz", + "integrity": "sha512-1OqefPDxGrlMwcbfpsTVRyzwdhr4W0uxYQzeA2F1CBc8WG04udg2+ybRnvh3XYL4TdHQrCahLtax2jc8xaE6rA==", + "requires": { + "style-to-object": "0.3.0" + } + }, + "style-to-object": { + "version": "0.3.0", + "resolved": "https://registry.npmmirror.com/style-to-object/-/style-to-object-0.3.0.tgz", + "integrity": "sha512-CzFnRRXhzWIdItT3OmF8SQfWyahHhjq3HwcMNCNLn+N7klOOqPjMeG/4JSu77D7ypZdGvSzvkrbyeTMizz2VrA==", + "requires": { + "inline-style-parser": "0.1.1" + } + }, + "stylehacks": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/stylehacks/-/stylehacks-5.1.0.tgz", + "integrity": "sha512-SzLmvHQTrIWfSgljkQCw2++C9+Ne91d/6Sp92I8c5uHTcy/PgeHamwITIbBW9wnFTY/3ZfSXR9HIL6Ikqmcu6Q==", + "requires": { + "browserslist": "^4.16.6", + "postcss-selector-parser": "^6.0.4" + } + }, + "stylis": { + "version": "4.0.13", + "resolved": "https://registry.npmmirror.com/stylis/-/stylis-4.0.13.tgz", + "integrity": "sha512-xGPXiFVl4YED9Jh7Euv2V220mriG9u4B2TA6Ybjc1catrstKD2PpIdU3U0RKpkVBC2EhmL/F0sPCr9vrFTNRag==" + }, + "subscriptions-transport-ws": { + "version": "0.9.19", + "resolved": "https://registry.npmjs.org/subscriptions-transport-ws/-/subscriptions-transport-ws-0.9.19.tgz", + "integrity": "sha512-dxdemxFFB0ppCLg10FTtRqH/31FNRL1y1BQv8209MK5I4CwALb7iihQg+7p65lFcIl8MHatINWBLOqpgU4Kyyw==", + "requires": { + "backo2": "^1.0.2", + "eventemitter3": "^3.1.0", + "iterall": "^1.2.1", + "symbol-observable": "^1.0.4", + "ws": "^5.2.0 || ^6.0.0 || ^7.0.0" + } + }, + "sudo-prompt": { + "version": "8.2.5", + "resolved": "https://registry.npmjs.org/sudo-prompt/-/sudo-prompt-8.2.5.tgz", + "integrity": "sha512-rlBo3HU/1zAJUrkY6jNxDOC9eVYliG6nS4JA8u8KAshITd07tafMc/Br7xQwCSseXwJ2iCcHCE8SNWX3q8Z+kw==" + }, + "supports-color": { + "version": "5.5.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", + "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", + "requires": { + "has-flag": "^3.0.0" + } + }, + "supports-preserve-symlinks-flag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz", + "integrity": "sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w==" + }, + "svgo": { + "version": "2.8.0", + "resolved": "https://registry.npmjs.org/svgo/-/svgo-2.8.0.tgz", + "integrity": "sha512-+N/Q9kV1+F+UeWYoSiULYo4xYSDQlTgb+ayMobAXPwMnLvop7oxKMo9OzIrX5x3eS4L4f2UHhc9axXwY8DpChg==", + "requires": { + "@trysound/sax": "0.2.0", + "commander": "^7.2.0", + "css-select": "^4.1.3", + "css-tree": "^1.1.3", + "csso": "^4.2.0", + "picocolors": "^1.0.0", + "stable": "^0.1.8" + } + }, + "swap-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/swap-case/-/swap-case-2.0.2.tgz", + "integrity": "sha512-kc6S2YS/2yXbtkSMunBtKdah4VFETZ8Oh6ONSmSd9bRxhqTrtARUCBUiWXH3xVPpvR7tz2CSnkuXVE42EcGnMw==", + "requires": { + "tslib": "^2.0.3" + } + }, + "symbol-observable": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/symbol-observable/-/symbol-observable-1.2.0.tgz", + "integrity": "sha512-e900nM8RRtGhlV36KGEU9k65K3mPb1WV70OdjfxlG2EAuM1noi/E/BaW/uMhL7bPEssK8QV57vN3esixjUvcXQ==" + }, + "sync-fetch": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/sync-fetch/-/sync-fetch-0.3.0.tgz", + "integrity": "sha512-dJp4qg+x4JwSEW1HibAuMi0IIrBI3wuQr2GimmqB7OXR50wmwzfdusG+p39R9w3R6aFtZ2mzvxvWKQ3Bd/vx3g==", + "requires": { + "buffer": "^5.7.0", + "node-fetch": "^2.6.1" + } + }, + "table": { + "version": "6.8.0", + "resolved": "https://registry.npmjs.org/table/-/table-6.8.0.tgz", + "integrity": "sha512-s/fitrbVeEyHKFa7mFdkuQMWlH1Wgw/yEXMt5xACT4ZpzWFluehAxRtUUQKPuWhaLAWhFcVx6w3oC8VKaUfPGA==", + "requires": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + }, + "dependencies": { + "ajv": { + "version": "8.11.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.11.0.tgz", + "integrity": "sha512-wGgprdCvMalC0BztXvitD2hC04YffAvtsUn93JbGXYLAtCUO4xd17mCCZQxUOItiBwZvJScWo8NIvQMQ71rdpg==", + "requires": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + } + }, + "json-schema-traverse": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz", + "integrity": "sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug==" + } + } + }, + "tapable": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-2.2.1.tgz", + "integrity": "sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ==" + }, + "tar": { + "version": "6.1.11", + "resolved": "https://registry.npmmirror.com/tar/-/tar-6.1.11.tgz", + "integrity": "sha512-an/KZQzQUkZCkuoAA64hM92X0Urb6VpRhAFllDzz44U2mcD5scmT3zBc4VgVpkugF580+DQn8eAFSyoQt0tznA==", + "requires": { + "chownr": "^2.0.0", + "fs-minipass": "^2.0.0", + "minipass": "^3.0.0", + "minizlib": "^2.1.1", + "mkdirp": "^1.0.3", + "yallist": "^4.0.0" + }, + "dependencies": { + "chownr": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/chownr/-/chownr-2.0.0.tgz", + "integrity": "sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ==" + }, + "mkdirp": { + "version": "1.0.4", + "resolved": "https://registry.npmmirror.com/mkdirp/-/mkdirp-1.0.4.tgz", + "integrity": "sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw==" + } + } + }, + "tar-fs": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/tar-fs/-/tar-fs-2.1.1.tgz", + "integrity": "sha512-V0r2Y9scmbDRLCNex/+hYzvp/zyYjvFbHPNgVTKfQvVrb6guiE/fxP+XblDNR011utopbkex2nM4dHNV6GDsng==", + "requires": { + "chownr": "^1.1.1", + "mkdirp-classic": "^0.5.2", + "pump": "^3.0.0", + "tar-stream": "^2.1.4" + } + }, + "tar-stream": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/tar-stream/-/tar-stream-2.2.0.tgz", + "integrity": "sha512-ujeqbceABgwMZxEJnk2HDY2DlnUZ+9oEcb1KzTVfYHio0UE6dG71n60d8D2I4qNvleWrrXpmjpt7vZeF1LnMZQ==", + "requires": { + "bl": "^4.0.3", + "end-of-stream": "^1.4.1", + "fs-constants": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^3.1.1" + } + }, + "term-size": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/term-size/-/term-size-2.2.1.tgz", + "integrity": "sha512-wK0Ri4fOGjv/XPy8SBHZChl8CM7uMc5VML7SqiQ0zG7+J5Vr+RMQDoHa2CNT6KHUnTGIXH34UDMkPzAUyapBZg==" + }, + "terser": { + "version": "5.13.1", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.13.1.tgz", + "integrity": "sha512-hn4WKOfwnwbYfe48NgrQjqNOH9jzLqRcIfbYytOXCOv46LBfWr9bDS17MQqOi+BWGD0sJK3Sj5NC/gJjiojaoA==", + "requires": { + "acorn": "^8.5.0", + "commander": "^2.20.0", + "source-map": "~0.8.0-beta.0", + "source-map-support": "~0.5.20" + }, + "dependencies": { + "acorn": { + "version": "8.7.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.7.1.tgz", + "integrity": "sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==" + }, + "commander": { + "version": "2.20.3", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", + "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + }, + "source-map": { + "version": "0.8.0-beta.0", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.8.0-beta.0.tgz", + "integrity": "sha512-2ymg6oRBpebeZi9UUNsgQ89bhx01TcTkmNTGnNO88imTmbSgy4nfujrgVEFKWpMTEGA11EDkTt7mqObTPdigIA==", + "requires": { + "whatwg-url": "^7.0.0" + } + }, + "tr46": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-1.0.1.tgz", + "integrity": "sha1-qLE/1r/SSJUZZ0zN5VujaTtwbQk=", + "requires": { + "punycode": "^2.1.0" + } + }, + "webidl-conversions": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-4.0.2.tgz", + "integrity": "sha512-YQ+BmxuTgd6UXZW3+ICGfyqRyHXVlD5GtQr5+qjiNW7bF0cqrzX500HVXPBOvgXb5YnzDd+h0zqyv61KUD7+Sg==" + }, + "whatwg-url": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-7.1.0.tgz", + "integrity": "sha512-WUu7Rg1DroM7oQvGWfOiAK21n74Gg+T4elXEQYkOhtyLeWiJFoOGLXPKI/9gzIie9CtwVLm8wtw6YJdKyxSjeg==", + "requires": { + "lodash.sortby": "^4.7.0", + "tr46": "^1.0.1", + "webidl-conversions": "^4.0.2" + } + } + } + }, + "terser-webpack-plugin": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.1.tgz", + "integrity": "sha512-GvlZdT6wPQKbDNW/GDQzZFg/j4vKU96yl2q6mcUkzKOgW4gwf1Z8cZToUCrz31XHlPWH8MVb1r2tFtdDtTGJ7g==", + "requires": { + "jest-worker": "^27.4.5", + "schema-utils": "^3.1.1", + "serialize-javascript": "^6.0.0", + "source-map": "^0.6.1", + "terser": "^5.7.2" + }, + "dependencies": { + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==" + }, + "jest-worker": { + "version": "27.5.1", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-27.5.1.tgz", + "integrity": "sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg==", + "requires": { + "@types/node": "*", + "merge-stream": "^2.0.0", + "supports-color": "^8.0.0" + } + }, + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + }, + "serialize-javascript": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", + "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "requires": { + "randombytes": "^2.1.0" + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + }, + "supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "requires": { + "has-flag": "^4.0.0" + } + } + } + }, + "text-table": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/text-table/-/text-table-0.2.0.tgz", + "integrity": "sha1-f17oI66AUgfACvLfSoTsP8+lcLQ=" + }, + "through": { + "version": "2.3.8", + "resolved": "https://registry.npmjs.org/through/-/through-2.3.8.tgz", + "integrity": "sha1-DdTJ/6q8NXlgsbckEV1+Doai4fU=" + }, + "through2": { + "version": "0.4.2", + "resolved": "https://registry.npmmirror.com/through2/-/through2-0.4.2.tgz", + "integrity": "sha512-45Llu+EwHKtAZYTPPVn3XZHBgakWMN3rokhEv5hu596XP+cNgplMg+Gj+1nmAvj+L0K7+N49zBKx5rah5u0QIQ==", + "requires": { + "readable-stream": "~1.0.17", + "xtend": "~2.1.1" + }, + "dependencies": { + "isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmmirror.com/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==" + }, + "object-keys": { + "version": "0.4.0", + "resolved": "https://registry.npmmirror.com/object-keys/-/object-keys-0.4.0.tgz", + "integrity": "sha512-ncrLw+X55z7bkl5PnUvHwFK9FcGuFYo9gtjws2XtSzL+aZ8tm830P60WJ0dSmFVaSalWieW5MD7kEdnXda9yJw==" + }, + "readable-stream": { + "version": "1.0.34", + "resolved": "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.0.34.tgz", + "integrity": "sha512-ok1qVCJuRkNmvebYikljxJA/UEsKwLl2nI1OmaqAu4/UE+h0wKCHok4XkL/gvi39OacXvw59RJUOFUkDib2rHg==", + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + } + }, + "string_decoder": { + "version": "0.10.31", + "resolved": "https://registry.npmmirror.com/string_decoder/-/string_decoder-0.10.31.tgz", + "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==" + }, + "xtend": { + "version": "2.1.2", + "resolved": "https://registry.npmmirror.com/xtend/-/xtend-2.1.2.tgz", + "integrity": "sha512-vMNKzr2rHP9Dp/e1NQFnLQlwlhp9L/LfvnsVdHxN1f+uggyVI3i08uD14GPvCToPkdsRfyPqIyYGmIk58V98ZQ==", + "requires": { + "object-keys": "~0.4.0" + } + } + } + }, + "timers-ext": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/timers-ext/-/timers-ext-0.1.7.tgz", + "integrity": "sha512-b85NUNzTSdodShTIbky6ZF02e8STtVVfD+fu4aXXShEELpozH+bCpJLYMPZbsABN2wDH7fJpqIoXxJpzbf0NqQ==", + "requires": { + "es5-ext": "~0.10.46", + "next-tick": "1" + } + }, + "timm": { + "version": "1.7.1", + "resolved": "https://registry.npmjs.org/timm/-/timm-1.7.1.tgz", + "integrity": "sha512-IjZc9KIotudix8bMaBW6QvMuq64BrJWFs1+4V0lXwWGQZwH+LnX87doAYhem4caOEusRP9/g6jVDQmZ8XOk1nw==" + }, + "tinycolor2": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/tinycolor2/-/tinycolor2-1.4.2.tgz", + "integrity": "sha512-vJhccZPs965sV/L2sU4oRQVAos0pQXwsvTLkWYdqJ+a8Q5kPFzJTuOFwy7UniPli44NKQGAglksjvOcpo95aZA==" + }, + "title-case": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/title-case/-/title-case-3.0.3.tgz", + "integrity": "sha512-e1zGYRvbffpcHIrnuqT0Dh+gEJtDaxDSoG4JAIpq4oDFyooziLBIiYQv0GBT4FUAnUop5uZ1hiIAj7oAF6sOCA==", + "requires": { + "tslib": "^2.0.3" + } + }, + "tmp": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.2.1.tgz", + "integrity": "sha512-76SUhtfqR2Ijn+xllcI5P1oyannHNHByD80W1q447gU3mp9G9PSpGdWmjUOHRDPiHYacIk66W7ubDTuPF3BEtQ==", + "requires": { + "rimraf": "^3.0.0" + } + }, + "to-fast-properties": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/to-fast-properties/-/to-fast-properties-2.0.0.tgz", + "integrity": "sha1-3F5pjL0HkmW8c+A3doGk5Og/YW4=" + }, + "to-readable-stream": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/to-readable-stream/-/to-readable-stream-1.0.0.tgz", + "integrity": "sha512-Iq25XBt6zD5npPhlLVXGFN3/gyR2/qODcKNNyTMd4vbm39HUaOiAM4PMq0eMVC/Tkxz+Zjdsc55g9yyz+Yq00Q==" + }, + "to-regex-range": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", + "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", + "requires": { + "is-number": "^7.0.0" + } + }, + "toidentifier": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/toidentifier/-/toidentifier-1.0.1.tgz", + "integrity": "sha512-o5sSPKEkg/DIQNmH43V0/uerLrpzVedkUh8tGNvaeXpfpuwjKenlSox/2O/BTlZUtEe+JG7s5YhEz608PlAHRA==" + }, + "token-types": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/token-types/-/token-types-4.2.0.tgz", + "integrity": "sha512-P0rrp4wUpefLncNamWIef62J0v0kQR/GfDVji9WKY7GDCWy5YbVSrKUTam07iWPZQGy0zWNOfstYTykMmPNR7w==", + "requires": { + "@tokenizer/token": "^0.3.0", + "ieee754": "^1.2.1" + } + }, + "tough-cookie": { + "version": "2.5.0", + "resolved": "https://registry.npmmirror.com/tough-cookie/-/tough-cookie-2.5.0.tgz", + "integrity": "sha512-nlLsUzgm1kfLXSXfRZMc1KLAugd4hqJHDTvc2hDIwS3mZAfMEuMbc03SujMF+GEcpaX/qboeycw6iO8JwVv2+g==", + "requires": { + "psl": "^1.1.28", + "punycode": "^2.1.1" + } + }, + "tr46": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", + "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" + }, + "trim-newlines": { + "version": "3.0.1", + "resolved": "https://registry.npmmirror.com/trim-newlines/-/trim-newlines-3.0.1.tgz", + "integrity": "sha512-c1PTsA3tYrIsLGkJkzHF+w9F2EyxfXGo4UyJc4pFL++FMjnq0HJS69T3M7d//gKrFKwy429bouPescbjecU+Zw==" + }, + "trim-repeated": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/trim-repeated/-/trim-repeated-1.0.0.tgz", + "integrity": "sha1-42RqLqTokTEr9+rObPsFOAvAHCE=", + "requires": { + "escape-string-regexp": "^1.0.2" + } + }, + "trough": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/trough/-/trough-1.0.5.tgz", + "integrity": "sha512-rvuRbTarPXmMb79SmzEp8aqXNKcK+y0XaB298IXueQ8I2PsrATcPBCSPyK/dDNa2iWOhKlfNnOjdAOTBU/nkFA==" + }, + "true-case-path": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/true-case-path/-/true-case-path-2.2.1.tgz", + "integrity": "sha512-0z3j8R7MCjy10kc/g+qg7Ln3alJTodw9aDuVWZa3uiWqfuBMKeAeP2ocWcxoyM3D73yz3Jt/Pu4qPr4wHSdB/Q==" + }, + "ts-node": { + "version": "9.1.1", + "resolved": "https://registry.npmjs.org/ts-node/-/ts-node-9.1.1.tgz", + "integrity": "sha512-hPlt7ZACERQGf03M253ytLY3dHbGNGrAq9qIHWUY9XHYl1z7wYngSr3OQ5xmui8o2AaxsONxIzjafLUiWBo1Fg==", + "requires": { + "arg": "^4.1.0", + "create-require": "^1.1.0", + "diff": "^4.0.1", + "make-error": "^1.1.1", + "source-map-support": "^0.5.17", + "yn": "3.1.1" + } + }, + "tsconfig-paths": { + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/tsconfig-paths/-/tsconfig-paths-3.14.1.tgz", + "integrity": "sha512-fxDhWnFSLt3VuTwtvJt5fpwxBHg5AdKWMsgcPOOIilyjymcYVZoCQF8fvFRezCNfblEXmi+PcM1eYHeOAgXCOQ==", + "requires": { + "@types/json5": "^0.0.29", + "json5": "^1.0.1", + "minimist": "^1.2.6", + "strip-bom": "^3.0.0" + }, + "dependencies": { + "json5": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "requires": { + "minimist": "^1.2.0" + } + } + } + }, + "tslib": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.3.1.tgz", + "integrity": "sha512-77EbyPPpMz+FRFRuAFlWMtmgUWGe9UOG2Z25NqCwiIjRhOf5iKGuzSe5P2w1laq+FkRy4p+PCuVkJSGkzTEKVw==" + }, + "tsutils": { + "version": "3.21.0", + "resolved": "https://registry.npmjs.org/tsutils/-/tsutils-3.21.0.tgz", + "integrity": "sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA==", + "requires": { + "tslib": "^1.8.1" + }, + "dependencies": { + "tslib": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.14.1.tgz", + "integrity": "sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==" + } + } + }, + "tunnel-agent": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/tunnel-agent/-/tunnel-agent-0.6.0.tgz", + "integrity": "sha1-J6XeoGs2sEoKmWZ3SykIaPD8QP0=", + "requires": { + "safe-buffer": "^5.0.1" + } + }, + "tweetnacl": { + "version": "0.14.5", + "resolved": "https://registry.npmmirror.com/tweetnacl/-/tweetnacl-0.14.5.tgz", + "integrity": "sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==" + }, + "type": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/type/-/type-1.2.0.tgz", + "integrity": "sha512-+5nt5AAniqsCnu2cEQQdpzCAh33kVx8n0VoFidKpB1dVVLAN/F+bgVOqOJqOnEnrhp222clB5p3vUlD+1QAnfg==" + }, + "type-check": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", + "integrity": "sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew==", + "requires": { + "prelude-ls": "^1.2.1" + } + }, + "type-fest": { + "version": "0.20.2", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", + "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==" + }, + "type-is": { + "version": "1.6.18", + "resolved": "https://registry.npmjs.org/type-is/-/type-is-1.6.18.tgz", + "integrity": "sha512-TkRKr9sUTxEH8MdfuCSP7VizJyzRNMjj2J2do2Jr3Kym598JVdEksuzPQCnlFPW4ky9Q+iA+ma9BGm06XQBy8g==", + "requires": { + "media-typer": "0.3.0", + "mime-types": "~2.1.24" + } + }, + "type-of": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/type-of/-/type-of-2.0.1.tgz", + "integrity": "sha1-5yoXQYllaOn2KDeNgW1pEvfyOXI=" + }, + "typedarray": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/typedarray/-/typedarray-0.0.6.tgz", + "integrity": "sha1-hnrHTjhkGHsdPUfZlqeOxciDB3c=" + }, + "typedarray-to-buffer": { + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz", + "integrity": "sha512-zdu8XMNEDepKKR+XYOXAVPtWui0ly0NtohUscw+UmaHiAWT8hrV1rr//H6V+0DvJ3OQ19S979M0laLfX8rm82Q==", + "requires": { + "is-typedarray": "^1.0.0" + } + }, + "ua-parser-js": { + "version": "0.7.31", + "resolved": "https://registry.npmjs.org/ua-parser-js/-/ua-parser-js-0.7.31.tgz", + "integrity": "sha512-qLK/Xe9E2uzmYI3qLeOmI0tEOt+TBBQyUIAh4aAgU05FVYzeZrKUdkAZfBNVGRaHVgV0TDkdEngJSw/SyQchkQ==" + }, + "unbox-primitive": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/unbox-primitive/-/unbox-primitive-1.0.2.tgz", + "integrity": "sha512-61pPlCD9h51VoreyJ0BReideM3MDKMKnh6+V9L08331ipq6Q8OFXZYiqP6n/tbHx4s5I9uRhcye6BrbkizkBDw==", + "requires": { + "call-bind": "^1.0.2", + "has-bigints": "^1.0.2", + "has-symbols": "^1.0.3", + "which-boxed-primitive": "^1.0.2" + } + }, + "unc-path-regex": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/unc-path-regex/-/unc-path-regex-0.1.2.tgz", + "integrity": "sha1-5z3T17DXxe2G+6xrCufYxqadUPo=" + }, + "underscore.string": { + "version": "3.3.6", + "resolved": "https://registry.npmmirror.com/underscore.string/-/underscore.string-3.3.6.tgz", + "integrity": "sha512-VoC83HWXmCrF6rgkyxS9GHv8W9Q5nhMKho+OadDJGzL2oDYbYEppBaCMH6pFlwLeqj2QS+hhkw2kpXkSdD1JxQ==", + "requires": { + "sprintf-js": "^1.1.1", + "util-deprecate": "^1.0.2" + }, + "dependencies": { + "sprintf-js": { + "version": "1.1.2", + "resolved": "https://registry.npmmirror.com/sprintf-js/-/sprintf-js-1.1.2.tgz", + "integrity": "sha512-VE0SOVEHCk7Qc8ulkWw3ntAzXuqf7S2lvwQaDLRnUeIEaKNQJzV6BwmLKhOqT61aGhfUMrXeaBk+oDGCzvhcug==" + } + } + }, + "unherit": { + "version": "1.1.3", + "resolved": "https://registry.npmmirror.com/unherit/-/unherit-1.1.3.tgz", + "integrity": "sha512-Ft16BJcnapDKp0+J/rqFC3Rrk6Y/Ng4nzsC028k2jdDII/rdZ7Wd3pPT/6+vIIxRagwRc9K0IUX0Ra4fKvw+WQ==", + "requires": { + "inherits": "^2.0.0", + "xtend": "^4.0.0" + } + }, + "unicode-canonical-property-names-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz", + "integrity": "sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ==" + }, + "unicode-match-property-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz", + "integrity": "sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q==", + "requires": { + "unicode-canonical-property-names-ecmascript": "^2.0.0", + "unicode-property-aliases-ecmascript": "^2.0.0" + } + }, + "unicode-match-property-value-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.0.0.tgz", + "integrity": "sha512-7Yhkc0Ye+t4PNYzOGKedDhXbYIBe1XEQYQxOPyhcXNMJ0WCABqqj6ckydd6pWRZTHV4GuCPKdBAUiMc60tsKVw==" + }, + "unicode-property-aliases-ecmascript": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.0.0.tgz", + "integrity": "sha512-5Zfuy9q/DFr4tfO7ZPeVXb1aPoeQSdeFMLpYuFebehDAhbuevLs5yxSZmIFN1tP5F9Wl4IpJrYojg85/zgyZHQ==" + }, + "unified": { + "version": "9.2.2", + "resolved": "https://registry.npmmirror.com/unified/-/unified-9.2.2.tgz", + "integrity": "sha512-Sg7j110mtefBD+qunSLO1lqOEKdrwBFBrR6Qd8f4uwkhWNlbkaqwHse6e7QvD3AP/MNoJdEDLaf8OxYyoWgorQ==", + "requires": { + "bail": "^1.0.0", + "extend": "^3.0.0", + "is-buffer": "^2.0.0", + "is-plain-obj": "^2.0.0", + "trough": "^1.0.0", + "vfile": "^4.0.0" + }, + "dependencies": { + "is-plain-obj": { + "version": "2.1.0", + "resolved": "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-2.1.0.tgz", + "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==" + } + } + }, + "unique-string": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unique-string/-/unique-string-2.0.0.tgz", + "integrity": "sha512-uNaeirEPvpZWSgzwsPGtU2zVSTrn/8L5q/IexZmH0eH6SA73CmAA5U4GwORTxQAZs95TAXLNqeLoPPNO5gZfWg==", + "requires": { + "crypto-random-string": "^2.0.0" + } + }, + "unist-builder": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-builder/-/unist-builder-2.0.3.tgz", + "integrity": "sha512-f98yt5pnlMWlzP539tPc4grGMsFaQQlP/vM396b00jngsiINumNmsY8rkXjfoi1c6QaM8nQ3vaGDuoKWbe/1Uw==" + }, + "unist-util-generated": { + "version": "1.1.6", + "resolved": "https://registry.npmmirror.com/unist-util-generated/-/unist-util-generated-1.1.6.tgz", + "integrity": "sha512-cln2Mm1/CZzN5ttGK7vkoGw+RZ8VcUH6BtGbq98DDtRGquAAOXig1mrBQYelOwMXYS8rK+vZDyyojSjp7JX+Lg==" + }, + "unist-util-is": { + "version": "4.1.0", + "resolved": "https://registry.npmmirror.com/unist-util-is/-/unist-util-is-4.1.0.tgz", + "integrity": "sha512-ZOQSsnce92GrxSqlnEEseX0gi7GH9zTJZ0p9dtu87WRb/37mMPO2Ilx1s/t9vBHrFhbgweUwb+t7cIn5dxPhZg==" + }, + "unist-util-modify-children": { + "version": "2.0.0", + "resolved": "https://registry.npmmirror.com/unist-util-modify-children/-/unist-util-modify-children-2.0.0.tgz", + "integrity": "sha512-HGrj7JQo9DwZt8XFsX8UD4gGqOsIlCih9opG6Y+N11XqkBGKzHo8cvDi+MfQQgiZ7zXRUiQREYHhjOBHERTMdg==", + "requires": { + "array-iterate": "^1.0.0" + } + }, + "unist-util-position": { + "version": "3.1.0", + "resolved": "https://registry.npmmirror.com/unist-util-position/-/unist-util-position-3.1.0.tgz", + "integrity": "sha512-w+PkwCbYSFw8vpgWD0v7zRCl1FpY3fjDSQ3/N/wNd9Ffa4gPi8+4keqt99N3XW6F99t/mUzp2xAhNmfKWp95QA==" + }, + "unist-util-remove-position": { + "version": "3.0.0", + "resolved": "https://registry.npmmirror.com/unist-util-remove-position/-/unist-util-remove-position-3.0.0.tgz", + "integrity": "sha512-17kIOuolVuK16LMb9KyMJlqdfCtlfQY5FjY3Sdo9iC7F5wqdXhNjMq0PBvMpkVNNnAmHxXssUW+rZ9T2zbP0Rg==", + "requires": { + "unist-util-visit": "^2.0.0" + } + }, + "unist-util-select": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/unist-util-select/-/unist-util-select-3.0.4.tgz", + "integrity": "sha512-xf1zCu4okgPqGLdhCDpRnjwBNyv3EqjiXRUbz2SdK1+qnLMB7uXXajfzuBvvbHoQ+JLyp4AEbFCGndmc6S72sw==", + "requires": { + "css-selector-parser": "^1.0.0", + "not": "^0.1.0", + "nth-check": "^2.0.0", + "unist-util-is": "^4.0.0", + "zwitch": "^1.0.0" + } + }, + "unist-util-stringify-position": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-util-stringify-position/-/unist-util-stringify-position-2.0.3.tgz", + "integrity": "sha512-3faScn5I+hy9VleOq/qNbAd6pAx7iH5jYBMS9I1HgQVijz/4mv5Bvw5iw1sC/90CODiKo81G/ps8AJrISn687g==", + "requires": { + "@types/unist": "^2.0.2" + } + }, + "unist-util-visit": { + "version": "2.0.3", + "resolved": "https://registry.npmmirror.com/unist-util-visit/-/unist-util-visit-2.0.3.tgz", + "integrity": "sha512-iJ4/RczbJMkD0712mGktuGpm/U4By4FfDonL7N/9tATGIF4imikjOuagyMY53tnZq3NP6BcmlrHhEKAfGWjh7Q==", + "requires": { + "@types/unist": "^2.0.0", + "unist-util-is": "^4.0.0", + "unist-util-visit-parents": "^3.0.0" + } + }, + "unist-util-visit-children": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/unist-util-visit-children/-/unist-util-visit-children-1.1.4.tgz", + "integrity": "sha512-sA/nXwYRCQVRwZU2/tQWUqJ9JSFM1X3x7JIOsIgSzrFHcfVt6NkzDtKzyxg2cZWkCwGF9CO8x4QNZRJRMK8FeQ==" + }, + "unist-util-visit-parents": { + "version": "3.1.1", + "resolved": "https://registry.npmmirror.com/unist-util-visit-parents/-/unist-util-visit-parents-3.1.1.tgz", + "integrity": "sha512-1KROIZWo6bcMrZEwiH2UrXDyalAa0uqzWCxCJj6lPOvTve2WkfgCytoDTPaMnodXh1WrXOq0haVYHj99ynJlsg==", + "requires": { + "@types/unist": "^2.0.0", + "unist-util-is": "^4.0.0" + } + }, + "universalify": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/universalify/-/universalify-2.0.0.tgz", + "integrity": "sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ==" + }, + "unixify": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/unixify/-/unixify-1.0.0.tgz", + "integrity": "sha1-OmQcjC/7zk2mg6XHDwOkYpQMIJA=", + "requires": { + "normalize-path": "^2.1.1" + }, + "dependencies": { + "normalize-path": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-2.1.1.tgz", + "integrity": "sha1-GrKLVW4Zg2Oowab35vogE3/mrtk=", + "requires": { + "remove-trailing-separator": "^1.0.1" + } + } + } + }, + "unpipe": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/unpipe/-/unpipe-1.0.0.tgz", + "integrity": "sha1-sr9O6FFKrmFltIF4KdIbLvSZBOw=" + }, + "unquote": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/unquote/-/unquote-1.1.1.tgz", + "integrity": "sha1-j97XMk7G6IoP+LkF58CYzcCG1UQ=" + }, + "update-notifier": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/update-notifier/-/update-notifier-5.1.0.tgz", + "integrity": "sha512-ItnICHbeMh9GqUy31hFPrD1kcuZ3rpxDZbf4KUDavXwS0bW5m7SLbDQpGX3UYr072cbrF5hFUs3r5tUsPwjfHw==", + "requires": { + "boxen": "^5.0.0", + "chalk": "^4.1.0", + "configstore": "^5.0.1", + "has-yarn": "^2.1.0", + "import-lazy": "^2.1.0", + "is-ci": "^2.0.0", + "is-installed-globally": "^0.4.0", + "is-npm": "^5.0.0", + "is-yarn-global": "^0.3.0", + "latest-version": "^5.1.0", + "pupa": "^2.1.1", + "semver": "^7.3.4", + "semver-diff": "^3.1.1", + "xdg-basedir": "^4.0.0" + } + }, + "upper-case": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/upper-case/-/upper-case-2.0.2.tgz", + "integrity": "sha512-KgdgDGJt2TpuwBUIjgG6lzw2GWFRCW9Qkfkiv0DxqHHLYJHmtmdUIKcZd8rHgFSjopVTlw6ggzCm1b8MFQwikg==", + "requires": { + "tslib": "^2.0.3" + } + }, + "upper-case-first": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/upper-case-first/-/upper-case-first-2.0.2.tgz", + "integrity": "sha512-514ppYHBaKwfJRK/pNC6c/OxfGa0obSnAl106u97Ed0I625Nin96KAjttZF6ZL3e1XLtphxnqrOi9iWgm+u+bg==", + "requires": { + "tslib": "^2.0.3" + } + }, + "uri-js": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/uri-js/-/uri-js-4.4.1.tgz", + "integrity": "sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg==", + "requires": { + "punycode": "^2.1.0" + } + }, + "urix": { + "version": "0.1.0", + "resolved": "https://registry.npmmirror.com/urix/-/urix-0.1.0.tgz", + "integrity": "sha512-Am1ousAhSLBeB9cG/7k7r2R0zj50uDRlZHPGbazid5s9rlF1F/QKYObEKSIunSjIOkJZqwRRLpvewjEkM7pSqg==" + }, + "url-loader": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/url-loader/-/url-loader-4.1.1.tgz", + "integrity": "sha512-3BTV812+AVHHOJQO8O5MkWgZ5aosP7GnROJwvzLS9hWDj00lZ6Z0wNak423Lp9PBZN05N+Jk/N5Si8jRAlGyWA==", + "requires": { + "loader-utils": "^2.0.0", + "mime-types": "^2.1.27", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "url-parse-lax": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/url-parse-lax/-/url-parse-lax-3.0.0.tgz", + "integrity": "sha1-FrXK/Afb42dsGxmZF3gj1lA6yww=", + "requires": { + "prepend-http": "^2.0.0" + } + }, + "use-composed-ref": { + "version": "1.3.0", + "resolved": "https://registry.npmmirror.com/use-composed-ref/-/use-composed-ref-1.3.0.tgz", + "integrity": "sha512-GLMG0Jc/jiKov/3Ulid1wbv3r54K9HlMW29IWcDFPEqFkSO2nS0MuefWgMJpeHQ9YJeXDL3ZUF+P3jdXlZX/cQ==", + "requires": {} + }, + "use-isomorphic-layout-effect": { + "version": "1.1.2", + "resolved": "https://registry.npmmirror.com/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.1.2.tgz", + "integrity": "sha512-49L8yCO3iGT/ZF9QttjwLF/ZD9Iwto5LnH5LmEdk/6cFmXddqi2ulF0edxTwjj+7mqvpVVGQWvbXZdn32wRSHA==", + "requires": {} + }, + "use-latest": { + "version": "1.2.1", + "resolved": "https://registry.npmmirror.com/use-latest/-/use-latest-1.2.1.tgz", + "integrity": "sha512-xA+AVm/Wlg3e2P/JiItTziwS7FK92LWrDB0p+hgXloIMuVCeJJ8v6f0eeHyPZaJrM+usM1FkFfbNCrJGs8A/zw==", + "requires": { + "use-isomorphic-layout-effect": "^1.1.1" + } + }, + "utif": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/utif/-/utif-2.0.1.tgz", + "integrity": "sha512-Z/S1fNKCicQTf375lIP9G8Sa1H/phcysstNrrSdZKj1f9g58J4NMgb5IgiEZN9/nLMPDwF0W7hdOe9Qq2IYoLg==", + "requires": { + "pako": "^1.0.5" + } + }, + "util-deprecate": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", + "integrity": "sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8=" + }, + "util.promisify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/util.promisify/-/util.promisify-1.0.1.tgz", + "integrity": "sha512-g9JpC/3He3bm38zsLupWryXHoEcS22YHthuPQSJdMy6KNrzIRzWqcsHzD/WUnqe45whVou4VIsPew37DoXWNrA==", + "requires": { + "define-properties": "^1.1.3", + "es-abstract": "^1.17.2", + "has-symbols": "^1.0.1", + "object.getownpropertydescriptors": "^2.1.0" + } + }, + "utila": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/utila/-/utila-0.4.0.tgz", + "integrity": "sha1-ihagXURWV6Oupe7MWxKk+lN5dyw=" + }, + "utility-types": { + "version": "3.10.0", + "resolved": "https://registry.npmjs.org/utility-types/-/utility-types-3.10.0.tgz", + "integrity": "sha512-O11mqxmi7wMKCo6HKFt5AhO4BwY3VV68YU07tgxfz8zJTIxr4BpsezN49Ffwy9j3ZpwwJp4fkRwjRzq3uWE6Rg==" + }, + "utils-merge": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/utils-merge/-/utils-merge-1.0.1.tgz", + "integrity": "sha1-n5VxD1CiZ5R7LMwSR0HBAoQn5xM=" + }, + "uuid": { + "version": "8.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", + "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==" + }, + "v8-compile-cache": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/v8-compile-cache/-/v8-compile-cache-2.3.0.tgz", + "integrity": "sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA==" + }, + "valid-url": { + "version": "1.0.9", + "resolved": "https://registry.npmjs.org/valid-url/-/valid-url-1.0.9.tgz", + "integrity": "sha1-HBRHm0DxOXp1eC8RXkCGRHQzogA=" + }, + "validate-npm-package-license": { + "version": "3.0.4", + "resolved": "https://registry.npmmirror.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz", + "integrity": "sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew==", + "requires": { + "spdx-correct": "^3.0.0", + "spdx-expression-parse": "^3.0.0" + } + }, + "value-or-promise": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/value-or-promise/-/value-or-promise-1.0.11.tgz", + "integrity": "sha512-41BrgH+dIbCFXClcSapVs5M6GkENd3gQOJpEfPDNa71LsUGMXDL0jMWpI/Rh7WhX+Aalfz2TTS3Zt5pUsbnhLg==" + }, + "vary": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/vary/-/vary-1.1.2.tgz", + "integrity": "sha1-IpnwLG3tMNSllhsLn3RSShj2NPw=" + }, + "verror": { + "version": "1.10.0", + "resolved": "https://registry.npmmirror.com/verror/-/verror-1.10.0.tgz", + "integrity": "sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==", + "requires": { + "assert-plus": "^1.0.0", + "core-util-is": "1.0.2", + "extsprintf": "^1.2.0" + }, + "dependencies": { + "core-util-is": { + "version": "1.0.2", + "resolved": "https://registry.npmmirror.com/core-util-is/-/core-util-is-1.0.2.tgz", + "integrity": "sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==" + } + } + }, + "vfile": { + "version": "4.2.1", + "resolved": "https://registry.npmmirror.com/vfile/-/vfile-4.2.1.tgz", + "integrity": "sha512-O6AE4OskCG5S1emQ/4gl8zK586RqA3srz3nfK/Viy0UPToBc5Trp9BVFb1u0CjsKrAWwnpr4ifM/KBXPWwJbCA==", + "requires": { + "@types/unist": "^2.0.0", + "is-buffer": "^2.0.0", + "unist-util-stringify-position": "^2.0.0", + "vfile-message": "^2.0.0" + } + }, + "vfile-location": { + "version": "3.2.0", + "resolved": "https://registry.npmmirror.com/vfile-location/-/vfile-location-3.2.0.tgz", + "integrity": "sha512-aLEIZKv/oxuCDZ8lkJGhuhztf/BW4M+iHdCwglA/eWc+vtuRFJj8EtgceYFX4LRjOhCAAiNHsKGssC6onJ+jbA==" + }, + "vfile-message": { + "version": "2.0.4", + "resolved": "https://registry.npmmirror.com/vfile-message/-/vfile-message-2.0.4.tgz", + "integrity": "sha512-DjssxRGkMvifUOJre00juHoP9DPWuzjxKuMDrhNbk2TdaYYBNMStsNhEOt3idrtI12VQYM/1+iM0KOzXi4pxwQ==", + "requires": { + "@types/unist": "^2.0.0", + "unist-util-stringify-position": "^2.0.0" + } + }, + "warning": { + "version": "4.0.3", + "resolved": "https://registry.npmmirror.com/warning/-/warning-4.0.3.tgz", + "integrity": "sha512-rpJyN222KWIvHJ/F53XSZv0Zl/accqHR8et1kpaMTD/fLCRxtV8iX8czMzY7sVZupTI3zcUTg8eycS2kNF9l6w==", + "requires": { + "loose-envify": "^1.0.0" + } + }, + "watchpack": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.3.1.tgz", + "integrity": "sha512-x0t0JuydIo8qCNctdDrn1OzH/qDzk2+rdCOC3YzumZ42fiMqmQ7T3xQurykYMhYfHaPHTp4ZxAx2NfUo1K6QaA==", + "requires": { + "glob-to-regexp": "^0.4.1", + "graceful-fs": "^4.1.2" + } + }, + "weak-lru-cache": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/weak-lru-cache/-/weak-lru-cache-1.2.2.tgz", + "integrity": "sha512-DEAoo25RfSYMuTGc9vPJzZcZullwIqRDSI9LOy+fkCJPi6hykCnfKaXTuPBDuXAUcqHXyOgFtHNp/kB2FjYHbw==" + }, + "web-namespaces": { + "version": "1.1.4", + "resolved": "https://registry.npmmirror.com/web-namespaces/-/web-namespaces-1.1.4.tgz", + "integrity": "sha512-wYxSGajtmoP4WxfejAPIr4l0fVh+jeMXZb08wNc0tMg6xsfZXj3cECqIK0G7ZAqUq0PP8WlMDtaOGVBTAWztNw==" + }, + "webidl-conversions": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", + "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" + }, + "webpack": { + "version": "5.72.1", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.72.1.tgz", + "integrity": "sha512-dXG5zXCLspQR4krZVR6QgajnZOjW2K/djHvdcRaDQvsjV9z9vaW6+ja5dZOYbqBBjF6kGXka/2ZyxNdc+8Jung==", + "requires": { + "@types/eslint-scope": "^3.7.3", + "@types/estree": "^0.0.51", + "@webassemblyjs/ast": "1.11.1", + "@webassemblyjs/wasm-edit": "1.11.1", + "@webassemblyjs/wasm-parser": "1.11.1", + "acorn": "^8.4.1", + "acorn-import-assertions": "^1.7.6", + "browserslist": "^4.14.5", + "chrome-trace-event": "^1.0.2", + "enhanced-resolve": "^5.9.3", + "es-module-lexer": "^0.9.0", + "eslint-scope": "5.1.1", + "events": "^3.2.0", + "glob-to-regexp": "^0.4.1", + "graceful-fs": "^4.2.9", + "json-parse-even-better-errors": "^2.3.1", + "loader-runner": "^4.2.0", + "mime-types": "^2.1.27", + "neo-async": "^2.6.2", + "schema-utils": "^3.1.0", + "tapable": "^2.1.1", + "terser-webpack-plugin": "^5.1.3", + "watchpack": "^2.3.1", + "webpack-sources": "^3.2.3" + }, + "dependencies": { + "acorn": { + "version": "8.7.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.7.1.tgz", + "integrity": "sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==" + }, + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + }, + "webpack-sources": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/webpack-sources/-/webpack-sources-3.2.3.tgz", + "integrity": "sha512-/DyMEOrDgLKKIG0fmvtz+4dUX/3Ghozwgm6iPp8KRhvn+eQf9+Q7GWxVNMk3+uCPWfdXYC4ExGBckIXdFEfH1w==" + } + } + }, + "webpack-assets-manifest": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/webpack-assets-manifest/-/webpack-assets-manifest-5.1.0.tgz", + "integrity": "sha512-kPuTMEjBrqZQVJ5M6yXNBCEdFbQQn7p+loNXt8NOeDFaAbsNFWqqwR0YL1mfG5LbwhK5FLXWXpuK3GuIIZ46rg==", + "requires": { + "chalk": "^4.0", + "deepmerge": "^4.0", + "lockfile": "^1.0", + "lodash.get": "^4.0", + "lodash.has": "^4.0", + "schema-utils": "^3.0", + "tapable": "^2.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "webpack-dev-middleware": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/webpack-dev-middleware/-/webpack-dev-middleware-4.3.0.tgz", + "integrity": "sha512-PjwyVY95/bhBh6VUqt6z4THplYcsvQ8YNNBTBM873xLVmw8FLeALn0qurHbs9EmcfhzQis/eoqypSnZeuUz26w==", + "requires": { + "colorette": "^1.2.2", + "mem": "^8.1.1", + "memfs": "^3.2.2", + "mime-types": "^2.1.30", + "range-parser": "^1.2.1", + "schema-utils": "^3.0.0" + }, + "dependencies": { + "schema-utils": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-3.1.1.tgz", + "integrity": "sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw==", + "requires": { + "@types/json-schema": "^7.0.8", + "ajv": "^6.12.5", + "ajv-keywords": "^3.5.2" + } + } + } + }, + "webpack-merge": { + "version": "5.8.0", + "resolved": "https://registry.npmjs.org/webpack-merge/-/webpack-merge-5.8.0.tgz", + "integrity": "sha512-/SaI7xY0831XwP6kzuwhKWVKDP9t1QY1h65lAFLbZqMPIuYcD9QAW4u9STIbU9kaJbPBB/geU/gLr1wDjOhQ+Q==", + "requires": { + "clone-deep": "^4.0.1", + "wildcard": "^2.0.0" + } + }, + "webpack-sources": { + "version": "1.4.3", + "resolved": "https://registry.npmjs.org/webpack-sources/-/webpack-sources-1.4.3.tgz", + "integrity": "sha512-lgTS3Xhv1lCOKo7SA5TjKXMjpSM4sBjNV5+q2bqesbSPs5FjGmU6jjtBSkX9b4qW87vDIsCIlUPOEhbZrMdjeQ==", + "requires": { + "source-list-map": "^2.0.0", + "source-map": "~0.6.1" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" + } + } + }, + "webpack-stats-plugin": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/webpack-stats-plugin/-/webpack-stats-plugin-1.0.3.tgz", + "integrity": "sha512-tV/SQHl6lKfBahJcNDmz8JG1rpWPB9NEDQSMIoL74oVAotdxYljpgIsgLzgc1N9QrtA9KEA0moJVwQtNZv2aDA==" + }, + "webpack-virtual-modules": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/webpack-virtual-modules/-/webpack-virtual-modules-0.3.2.tgz", + "integrity": "sha512-RXQXioY6MhzM4CNQwmBwKXYgBs6ulaiQ8bkNQEl2J6Z+V+s7lgl/wGvaI/I0dLnYKB8cKsxQc17QOAVIphPLDw==", + "requires": { + "debug": "^3.0.0" + } + }, + "whatwg-url": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", + "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", + "requires": { + "tr46": "~0.0.3", + "webidl-conversions": "^3.0.0" + } + }, + "which": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/which/-/which-1.3.1.tgz", + "integrity": "sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ==", + "requires": { + "isexe": "^2.0.0" + } + }, + "which-boxed-primitive": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz", + "integrity": "sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg==", + "requires": { + "is-bigint": "^1.0.1", + "is-boolean-object": "^1.1.0", + "is-number-object": "^1.0.4", + "is-string": "^1.0.5", + "is-symbol": "^1.0.3" + } + }, + "which-module": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/which-module/-/which-module-2.0.0.tgz", + "integrity": "sha1-2e8H3Od7mQK4o6j6SzHD4/fm6Ho=" + }, + "wide-align": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.5.tgz", + "integrity": "sha512-eDMORYaPNZ4sQIuuYPDHdQvf4gyCF9rEEV/yPxGfwPkRodwEgiMUUXTx/dex+Me0wxx53S+NgUHaP7y3MGlDmg==", + "requires": { + "string-width": "^1.0.2 || 2 || 3 || 4" + } + }, + "widest-line": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/widest-line/-/widest-line-3.1.0.tgz", + "integrity": "sha512-NsmoXalsWVDMGupxZ5R08ka9flZjjiLvHVAWYOKtiKM8ujtZWr9cRffak+uSE48+Ob8ObalXpwyeUiyDD6QFgg==", + "requires": { + "string-width": "^4.0.0" + } + }, + "wildcard": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/wildcard/-/wildcard-2.0.0.tgz", + "integrity": "sha512-JcKqAHLPxcdb9KM49dufGXn2x3ssnfjbcaQdLlfZsL9rH9wgDQjUtDxbo8NE0F6SFvydeu1VhZe7hZuHsB2/pw==" + }, + "word-wrap": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/word-wrap/-/word-wrap-1.2.3.tgz", + "integrity": "sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ==" + }, + "workbox-background-sync": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-background-sync/-/workbox-background-sync-4.3.1.tgz", + "integrity": "sha512-1uFkvU8JXi7L7fCHVBEEnc3asPpiAL33kO495UMcD5+arew9IbKW2rV5lpzhoWcm/qhGB89YfO4PmB/0hQwPRg==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-broadcast-update": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-broadcast-update/-/workbox-broadcast-update-4.3.1.tgz", + "integrity": "sha512-MTSfgzIljpKLTBPROo4IpKjESD86pPFlZwlvVG32Kb70hW+aob4Jxpblud8EhNb1/L5m43DUM4q7C+W6eQMMbA==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-build": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-build/-/workbox-build-4.3.1.tgz", + "integrity": "sha512-UHdwrN3FrDvicM3AqJS/J07X0KXj67R8Cg0waq1MKEOqzo89ap6zh6LmaLnRAjpB+bDIz+7OlPye9iii9KBnxw==", + "requires": { + "@babel/runtime": "^7.3.4", + "@hapi/joi": "^15.0.0", + "common-tags": "^1.8.0", + "fs-extra": "^4.0.2", + "glob": "^7.1.3", + "lodash.template": "^4.4.0", + "pretty-bytes": "^5.1.0", + "stringify-object": "^3.3.0", + "strip-comments": "^1.0.2", + "workbox-background-sync": "^4.3.1", + "workbox-broadcast-update": "^4.3.1", + "workbox-cacheable-response": "^4.3.1", + "workbox-core": "^4.3.1", + "workbox-expiration": "^4.3.1", + "workbox-google-analytics": "^4.3.1", + "workbox-navigation-preload": "^4.3.1", + "workbox-precaching": "^4.3.1", + "workbox-range-requests": "^4.3.1", + "workbox-routing": "^4.3.1", + "workbox-strategies": "^4.3.1", + "workbox-streams": "^4.3.1", + "workbox-sw": "^4.3.1", + "workbox-window": "^4.3.1" + }, + "dependencies": { + "fs-extra": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-4.0.3.tgz", + "integrity": "sha512-q6rbdDd1o2mAnQreO7YADIxf/Whx4AHBiRf6d+/cVT8h44ss+lHgxf1FemcqDnQt9X3ct4McHr+JMGlYSsK7Cg==", + "requires": { + "graceful-fs": "^4.1.2", + "jsonfile": "^4.0.0", + "universalify": "^0.1.0" + } + }, + "jsonfile": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-4.0.0.tgz", + "integrity": "sha1-h3Gq4HmbZAdrdmQPygWPnBDjPss=", + "requires": { + "graceful-fs": "^4.1.6" + } + }, + "universalify": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/universalify/-/universalify-0.1.2.tgz", + "integrity": "sha512-rBJeI5CXAlmy1pV+617WB9J63U6XcazHHF2f2dbJix4XzpUF0RS3Zbj0FGIOCAva5P/d/GBOYaACQ1w+0azUkg==" + } + } + }, + "workbox-cacheable-response": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-cacheable-response/-/workbox-cacheable-response-4.3.1.tgz", + "integrity": "sha512-Rp5qlzm6z8IOvnQNkCdO9qrDgDpoPNguovs0H8C+wswLuPgSzSp9p2afb5maUt9R1uTIwOXrVQMmPfPypv+npw==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-core": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-core/-/workbox-core-4.3.1.tgz", + "integrity": "sha512-I3C9jlLmMKPxAC1t0ExCq+QoAMd0vAAHULEgRZ7kieCdUd919n53WC0AfvokHNwqRhGn+tIIj7vcb5duCjs2Kg==" + }, + "workbox-expiration": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-expiration/-/workbox-expiration-4.3.1.tgz", + "integrity": "sha512-vsJLhgQsQouv9m0rpbXubT5jw0jMQdjpkum0uT+d9tTwhXcEZks7qLfQ9dGSaufTD2eimxbUOJfWLbNQpIDMPw==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-google-analytics": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-google-analytics/-/workbox-google-analytics-4.3.1.tgz", + "integrity": "sha512-xzCjAoKuOb55CBSwQrbyWBKqp35yg1vw9ohIlU2wTy06ZrYfJ8rKochb1MSGlnoBfXGWss3UPzxR5QL5guIFdg==", + "requires": { + "workbox-background-sync": "^4.3.1", + "workbox-core": "^4.3.1", + "workbox-routing": "^4.3.1", + "workbox-strategies": "^4.3.1" + } + }, + "workbox-navigation-preload": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-navigation-preload/-/workbox-navigation-preload-4.3.1.tgz", + "integrity": "sha512-K076n3oFHYp16/C+F8CwrRqD25GitA6Rkd6+qAmLmMv1QHPI2jfDwYqrytOfKfYq42bYtW8Pr21ejZX7GvALOw==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-precaching": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-precaching/-/workbox-precaching-4.3.1.tgz", + "integrity": "sha512-piSg/2csPoIi/vPpp48t1q5JLYjMkmg5gsXBQkh/QYapCdVwwmKlU9mHdmy52KsDGIjVaqEUMFvEzn2LRaigqQ==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-range-requests": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-range-requests/-/workbox-range-requests-4.3.1.tgz", + "integrity": "sha512-S+HhL9+iTFypJZ/yQSl/x2Bf5pWnbXdd3j57xnb0V60FW1LVn9LRZkPtneODklzYuFZv7qK6riZ5BNyc0R0jZA==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-routing": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-routing/-/workbox-routing-4.3.1.tgz", + "integrity": "sha512-FkbtrODA4Imsi0p7TW9u9MXuQ5P4pVs1sWHK4dJMMChVROsbEltuE79fBoIk/BCztvOJ7yUpErMKa4z3uQLX+g==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-strategies": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-strategies/-/workbox-strategies-4.3.1.tgz", + "integrity": "sha512-F/+E57BmVG8dX6dCCopBlkDvvhg/zj6VDs0PigYwSN23L8hseSRwljrceU2WzTvk/+BSYICsWmRq5qHS2UYzhw==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-streams": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-streams/-/workbox-streams-4.3.1.tgz", + "integrity": "sha512-4Kisis1f/y0ihf4l3u/+ndMkJkIT4/6UOacU3A4BwZSAC9pQ9vSvJpIi/WFGQRH/uPXvuVjF5c2RfIPQFSS2uA==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "workbox-sw": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-sw/-/workbox-sw-4.3.1.tgz", + "integrity": "sha512-0jXdusCL2uC5gM3yYFT6QMBzKfBr2XTk0g5TPAV4y8IZDyVNDyj1a8uSXy3/XrvkVTmQvLN4O5k3JawGReXr9w==" + }, + "workbox-window": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/workbox-window/-/workbox-window-4.3.1.tgz", + "integrity": "sha512-C5gWKh6I58w3GeSc0wp2Ne+rqVw8qwcmZnQGpjiek8A2wpbxSJb1FdCoQVO+jDJs35bFgo/WETgl1fqgsxN0Hg==", + "requires": { + "workbox-core": "^4.3.1" + } + }, + "wrap-ansi": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-6.2.0.tgz", + "integrity": "sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA==", + "requires": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + }, + "dependencies": { + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "requires": { + "color-convert": "^2.0.1" + } + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" + } + } + }, + "wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + }, + "write-file-atomic": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/write-file-atomic/-/write-file-atomic-3.0.3.tgz", + "integrity": "sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q==", + "requires": { + "imurmurhash": "^0.1.4", + "is-typedarray": "^1.0.0", + "signal-exit": "^3.0.2", + "typedarray-to-buffer": "^3.1.5" + } + }, + "ws": { + "version": "7.4.5", + "resolved": "https://registry.npmjs.org/ws/-/ws-7.4.5.tgz", + "integrity": "sha512-xzyu3hFvomRfXKH8vOFMU3OguG6oOvhXMo3xsGy3xWExqaM2dxBbVxuD99O7m3ZUFMvvscsZDqxfgMaRr/Nr1g==", + "requires": {} + }, + "xdg-basedir": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/xdg-basedir/-/xdg-basedir-4.0.0.tgz", + "integrity": "sha512-PSNhEJDejZYV7h50BohL09Er9VaIefr2LMAf3OEmpCkjOi34eYyQYAXUTjEQtZJTKcF0E2UKTh+osDLsgNim9Q==" + }, + "xhr": { + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/xhr/-/xhr-2.6.0.tgz", + "integrity": "sha512-/eCGLb5rxjx5e3mF1A7s+pLlR6CGyqWN91fv1JgER5mVWg1MZmlhBvy9kjcsOdRk8RrIujotWyJamfyrp+WIcA==", + "requires": { + "global": "~4.4.0", + "is-function": "^1.0.1", + "parse-headers": "^2.0.0", + "xtend": "^4.0.0" + } + }, + "xml-parse-from-string": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/xml-parse-from-string/-/xml-parse-from-string-1.0.1.tgz", + "integrity": "sha1-qQKekp09vN7RafPG4oI42VpdWig=" + }, + "xml2js": { + "version": "0.4.23", + "resolved": "https://registry.npmjs.org/xml2js/-/xml2js-0.4.23.tgz", + "integrity": "sha512-ySPiMjM0+pLDftHgXY4By0uswI3SPKLDw/i3UXbnO8M/p28zqexCUoPmQFrYD+/1BzhGJSs2i1ERWKJAtiLrug==", + "requires": { + "sax": ">=0.6.0", + "xmlbuilder": "~11.0.0" + } + }, + "xmlbuilder": { + "version": "11.0.1", + "resolved": "https://registry.npmjs.org/xmlbuilder/-/xmlbuilder-11.0.1.tgz", + "integrity": "sha512-fDlsI/kFEx7gLvbecc0/ohLG50fugQp8ryHzMTuW9vSa1GJ0XYWKnhsUx7oie3G98+r56aTQIUB4kht42R3JvA==" + }, + "xmlhttprequest-ssl": { + "version": "1.6.3", + "resolved": "https://registry.npmjs.org/xmlhttprequest-ssl/-/xmlhttprequest-ssl-1.6.3.tgz", + "integrity": "sha512-3XfeQE/wNkvrIktn2Kf0869fC0BN6UpydVasGIeSm2B1Llihf7/0UfZM+eCkOw3P7bP4+qPgqhm7ZoxuJtFU0Q==" + }, + "xss": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/xss/-/xss-1.0.11.tgz", + "integrity": "sha512-EimjrjThZeK2MO7WKR9mN5ZC1CSqivSl55wvUK5EtU6acf0rzEE1pN+9ZDrFXJ82BRp3JL38pPE6S4o/rpp1zQ==", + "requires": { + "commander": "^2.20.3", + "cssfilter": "0.0.10" + }, + "dependencies": { + "commander": { + "version": "2.20.3", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", + "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + } + } + }, + "xstate": { + "version": "4.32.0", + "resolved": "https://registry.npmjs.org/xstate/-/xstate-4.32.0.tgz", + "integrity": "sha512-62gETqwnw4pBRe+tVWMt8hLgWEU8lq2qO8VN5PWmTELceRVt3I1bu1cwdraVRHUn4Bb2lnhNzn1A73oShuC+8g==" + }, + "xtend": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/xtend/-/xtend-4.0.2.tgz", + "integrity": "sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ==" + }, + "xxhash-wasm": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/xxhash-wasm/-/xxhash-wasm-0.4.2.tgz", + "integrity": "sha512-/eyHVRJQCirEkSZ1agRSCwriMhwlyUcFkXD5TPVSLP+IPzjsqMVzZwdoczLp1SoQU0R3dxz1RpIK+4YNQbCVOA==" + }, + "y18n": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-4.0.3.tgz", + "integrity": "sha512-JKhqTOwSrqNA1NY5lSztJ1GrBiUodLMmIZuLiDaMRJ+itFd+ABVE8XBjOvIWL+rSqNDC74LCSFmlb/U4UZ4hJQ==" + }, + "yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + }, + "yaml": { + "version": "1.10.2", + "resolved": "https://registry.npmjs.org/yaml/-/yaml-1.10.2.tgz", + "integrity": "sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg==" + }, + "yaml-loader": { + "version": "0.6.0", + "resolved": "https://registry.npmjs.org/yaml-loader/-/yaml-loader-0.6.0.tgz", + "integrity": "sha512-1bNiLelumURyj+zvVHOv8Y3dpCri0F2S+DCcmps0pA1zWRLjS+FhZQg4o3aUUDYESh73+pKZNI18bj7stpReow==", + "requires": { + "loader-utils": "^1.4.0", + "yaml": "^1.8.3" + }, + "dependencies": { + "json5": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-1.0.1.tgz", + "integrity": "sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow==", + "requires": { + "minimist": "^1.2.0" + } + }, + "loader-utils": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-1.4.0.tgz", + "integrity": "sha512-qH0WSMBtn/oHuwjy/NucEgbx5dbxxnxup9s4PVXJUDHZBQY+s0NWA9rJf53RBnQZxfch7euUui7hpoAPvALZdA==", + "requires": { + "big.js": "^5.2.2", + "emojis-list": "^3.0.0", + "json5": "^1.0.1" + } + } + } + }, + "yargs": { + "version": "15.4.1", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-15.4.1.tgz", + "integrity": "sha512-aePbxDmcYW++PaqBsJ+HYUFwCdv4LVvdnhBy78E57PIor8/OVvhMrADFFEDh8DHDFRv/O9i3lPhsENjO7QX0+A==", + "requires": { + "cliui": "^6.0.0", + "decamelize": "^1.2.0", + "find-up": "^4.1.0", + "get-caller-file": "^2.0.1", + "require-directory": "^2.1.1", + "require-main-filename": "^2.0.0", + "set-blocking": "^2.0.0", + "string-width": "^4.2.0", + "which-module": "^2.0.0", + "y18n": "^4.0.0", + "yargs-parser": "^18.1.2" + } + }, + "yargs-parser": { + "version": "18.1.3", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-18.1.3.tgz", + "integrity": "sha512-o50j0JeToy/4K6OZcaQmW6lyXXKhq7csREXcDwk2omFPJEwUNOVtJKvmDr9EI1fAJZUyZcRF7kxGBWmRXudrCQ==", + "requires": { + "camelcase": "^5.0.0", + "decamelize": "^1.2.0" + } + }, + "yeast": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/yeast/-/yeast-0.1.2.tgz", + "integrity": "sha1-AI4G2AlDIMNy28L47XagymyKxBk=" + }, + "yn": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/yn/-/yn-3.1.1.tgz", + "integrity": "sha512-Ux4ygGWsu2c7isFWe8Yu1YluJmqVhxqK2cLXNQA5AcC3QfbGNpM7fu0Y8b/z16pXLnFxZYvWhd3fhBY9DLmC6Q==" + }, + "yocto-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", + "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==" + }, + "yoga-layout-prebuilt": { + "version": "1.10.0", + "resolved": "https://registry.npmjs.org/yoga-layout-prebuilt/-/yoga-layout-prebuilt-1.10.0.tgz", + "integrity": "sha512-YnOmtSbv4MTf7RGJMK0FvZ+KD8OEe/J5BNnR0GHhD8J/XcG/Qvxgszm0Un6FTHWW4uHlTgP0IztiXQnGyIR45g==", + "requires": { + "@types/yoga-layout": "1.9.2" + } + }, + "yurnalist": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/yurnalist/-/yurnalist-2.1.0.tgz", + "integrity": "sha512-PgrBqosQLM3gN2xBFIMDLACRTV9c365VqityKKpSTWpwR+U4LAFR3rSVyEoscWlu3EzX9+Y0I86GXUKxpHFl6w==", + "requires": { + "chalk": "^2.4.2", + "inquirer": "^7.0.0", + "is-ci": "^2.0.0", + "read": "^1.0.7", + "strip-ansi": "^5.2.0" + }, + "dependencies": { + "ansi-regex": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-4.1.1.tgz", + "integrity": "sha512-ILlv4k/3f6vfQ4OoP2AGvirOktlQ98ZEL1k9FaQjxa3L1abBgbuTDAdPOpvbGncC0BTVQrl+OM8xZGK6tWXt7g==" + }, + "chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "strip-ansi": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-5.2.0.tgz", + "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", + "requires": { + "ansi-regex": "^4.1.0" + } + } + } + }, + "zwitch": { + "version": "1.0.5", + "resolved": "https://registry.npmmirror.com/zwitch/-/zwitch-1.0.5.tgz", + "integrity": "sha512-V50KMwwzqJV0NpZIZFwfOD5/lyny3WlSzRiXgA0G7VUnRlqttta1L6UQIHzd6EuBY/cHGfwTIck7w1yH6Q5zUw==" + } + } +} diff --git a/website/package.json b/website/package.json new file mode 100644 index 0000000000..6337db69cb --- /dev/null +++ b/website/package.json @@ -0,0 +1,68 @@ +{ + "name": "gatsby-starter-default", + "private": true, + "description": "A simple starter to get up and developing quickly with Gatsby", + "version": "0.1.0", + "author": "Kyle Mathews ", + "dependencies": { + "@icon-park/react": "^1.3.5", + "@mantine/core": "^4.2.5", + "@mantine/hooks": "^4.2.5", + "antd": "^4.21.0", + "aos": "^2.3.4", + "autoprefixer": "^10.4.7", + "gatsby": "^4.17.2", + "gatsby-link": "^4.17.0", + "gatsby-plugin-gatsby-cloud": "^4.17.0", + "gatsby-plugin-image": "^2.17.0", + "gatsby-plugin-manifest": "^4.17.0", + "gatsby-plugin-mantine": "^4.0.0", + "gatsby-plugin-offline": "^5.17.0", + "gatsby-plugin-postcss": "^5.16.0", + "gatsby-plugin-react-helmet": "^5.17.0", + "gatsby-plugin-sass": "^5.14.0", + "gatsby-plugin-sharp": "^4.17.0", + "gatsby-react-router-scroll": "^5.17.0", + "gatsby-remark-autolink-headers": "^5.16.0", + "gatsby-remark-images": "^6.17.0", + "gatsby-remark-table-of-contents": "^2.0.0", + "gatsby-script": "^1.2.0", + "gatsby-source-filesystem": "^4.17.0", + "gatsby-transformer-json": "^4.15.0", + "gatsby-transformer-plaintext": "^2.0.0", + "gatsby-transformer-remark": "^5.14.0", + "gatsby-transformer-sharp": "^4.17.0", + "node-sass": "^6.0.1", + "postcss": "^8.4.14", + "prop-types": "^15.8.1", + "react": "^18.1.0", + "react-dom": "^18.1.0", + "react-helmet": "^6.1.0", + "react-highlight": "^0.14.0", + "react-json-view": "^1.21.3", + "tailwindcss": "^3.1.0" + }, + "devDependencies": { + "prettier": "^2.6.2" + }, + "keywords": [ + "gatsby" + ], + "license": "0BSD", + "scripts": { + "build": "gatsby build", + "develop": "gatsby develop", + "format": "prettier --write \"**/*.{js,jsx,ts,tsx,json,md,css}\"", + "start": "gatsby develop", + "serve": "gatsby serve", + "clean": "gatsby clean", + "test": "echo \"Write tests! -> https://gatsby.dev/unit-testing\" && exit 1" + }, + "repository": { + "type": "git", + "url": "https://github.com/gatsbyjs/gatsby-starter-default" + }, + "bugs": { + "url": "https://github.com/gatsbyjs/gatsby/issues" + } +} diff --git a/website/plugins/json-content/gatsby-node.js b/website/plugins/json-content/gatsby-node.js new file mode 100644 index 0000000000..e20704506e --- /dev/null +++ b/website/plugins/json-content/gatsby-node.js @@ -0,0 +1,35 @@ +const isNodeSupported = node => { + return isTextFile(node) || isFileWithoutExtension(node) +} + +const isTextFile = node => { + return node.internal.type === "File" && (node.internal.mediaType === "application/json" || node.internal.mediaType.includes("sql")) +} + +const isFileWithoutExtension = node => { + return node.internal.type === "File" && node.internal.mediaType === "application/octet-stream" && !node.internal.extension +} + +exports.onCreateNode = async ({ node, actions, loadNodeContent, createNodeId, createContentDigest }) => { + if (!isNodeSupported(node)) { + return + } + const { createNode, createParentChildLink } = actions + const content = await loadNodeContent(node) + const id = createNodeId(`${node.id} >>> jsonContent`) + const plainTextNode = { + id, + children: [], + content, + parent: node.id, + internal: { + contentDigest: createContentDigest(content), + type: "jsonContent", + }, + } + createNode(plainTextNode) + createParentChildLink({ + parent: node, + child: plainTextNode, + }) +} diff --git a/website/plugins/json-content/package.json b/website/plugins/json-content/package.json new file mode 100644 index 0000000000..d375461032 --- /dev/null +++ b/website/plugins/json-content/package.json @@ -0,0 +1,40 @@ +{ + "name": "json-content", + "version": "2.0.0", + "description": "", + "main": "index.js", + "author": "huang yi chao", + "license": "MIT", + "private": false, + "keywords": [ + "gatsby", + "gatsby-plugin", + "plaintext", + "text" + ], + "peerDependencies": { + "gatsby": "^3.0.0" + }, + "dependencies": { + "@babel/runtime": "^7.13.10" + }, + "devDependencies": { + "@babel/cli": "^7.13.14", + "@babel/core": "^7.13.15", + "babel-preset-gatsby-package": "^1.3.0", + "bluebird": "^3.7.2", + "cross-env": "^7.0.3", + "jest": "^26.6.3", + "prettier": "^2.2.1" + }, + "scripts": { + "build": "babel src --out-dir . --ignore **/__tests__", + "prepare": "cross-env NODE_ENV=production npm run build", + "watch": "babel -w src --out-dir . --ignore **/__tests__", + "test": "jest" + }, + "engines": { + "node": ">=10.13.0" + } + } + \ No newline at end of file diff --git a/website/plugins/json-content/yarn.lock b/website/plugins/json-content/yarn.lock new file mode 100644 index 0000000000..977ccf4641 --- /dev/null +++ b/website/plugins/json-content/yarn.lock @@ -0,0 +1,4490 @@ +# THIS IS AN AUTOGENERATED FILE. DO NOT EDIT THIS FILE DIRECTLY. +# yarn lockfile v1 + + +"@ampproject/remapping@^2.1.0": + version "2.2.0" + resolved "https://registry.yarnpkg.com/@ampproject/remapping/-/remapping-2.2.0.tgz#56c133824780de3174aed5ab6834f3026790154d" + integrity sha512-qRmjj8nj9qmLTQXXmaR1cck3UXSRMPrbsLJAasZpF+t3riI71BXed5ebIOYwQntykeZuhjsdweEc9BxH5Jc26w== + dependencies: + "@jridgewell/gen-mapping" "^0.1.0" + "@jridgewell/trace-mapping" "^0.3.9" + +"@babel/cli@^7.13.14": + version "7.17.10" + resolved "https://registry.yarnpkg.com/@babel/cli/-/cli-7.17.10.tgz#5ea0bf6298bb78f3b59c7c06954f9bd1c79d5943" + integrity sha512-OygVO1M2J4yPMNOW9pb+I6kFGpQK77HmG44Oz3hg8xQIl5L/2zq+ZohwAdSaqYgVwM0SfmPHZHphH4wR8qzVYw== + dependencies: + "@jridgewell/trace-mapping" "^0.3.8" + commander "^4.0.1" + convert-source-map "^1.1.0" + fs-readdir-recursive "^1.1.0" + glob "^7.0.0" + make-dir "^2.1.0" + slash "^2.0.0" + optionalDependencies: + "@nicolo-ribaudo/chokidar-2" "2.1.8-no-fsevents.3" + chokidar "^3.4.0" + +"@babel/code-frame@^7.0.0", "@babel/code-frame@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.16.7.tgz#44416b6bd7624b998f5b1af5d470856c40138789" + integrity sha512-iAXqUn8IIeBTNd72xsFlgaXHkMBMt6y4HJp1tIaK465CWLT/fG1aqB7ykr95gHHmlBdGbFeWWfyB4NJJ0nmeIg== + dependencies: + "@babel/highlight" "^7.16.7" + +"@babel/compat-data@^7.13.11", "@babel/compat-data@^7.17.10": + version "7.17.10" + resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.17.10.tgz#711dc726a492dfc8be8220028b1b92482362baab" + integrity sha512-GZt/TCsG70Ms19gfZO1tM4CVnXsPgEPBCpJu+Qz3L0LUDsY5nZqFZglIoPC1kIYOtNBZlrnFT+klg12vFGZXrw== + +"@babel/core@^7.1.0", "@babel/core@^7.12.3", "@babel/core@^7.13.15", "@babel/core@^7.7.5": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.18.2.tgz#87b2fcd7cce9becaa7f5acebdc4f09f3dd19d876" + integrity sha512-A8pri1YJiC5UnkdrWcmfZTJTV85b4UXTAfImGmCfYmax4TR9Cw8sDS0MOk++Gp2mE/BefVJ5nwy5yzqNJbP/DQ== + dependencies: + "@ampproject/remapping" "^2.1.0" + "@babel/code-frame" "^7.16.7" + "@babel/generator" "^7.18.2" + "@babel/helper-compilation-targets" "^7.18.2" + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helpers" "^7.18.2" + "@babel/parser" "^7.18.0" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.2" + "@babel/types" "^7.18.2" + convert-source-map "^1.7.0" + debug "^4.1.0" + gensync "^1.0.0-beta.2" + json5 "^2.2.1" + semver "^6.3.0" + +"@babel/generator@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.18.2.tgz#33873d6f89b21efe2da63fe554460f3df1c5880d" + integrity sha512-W1lG5vUwFvfMd8HVXqdfbuG7RuaSrTCCD8cl8fP8wOivdbtbIg2Db3IWUcgvfxKbbn6ZBGYRW/Zk1MIwK49mgw== + dependencies: + "@babel/types" "^7.18.2" + "@jridgewell/gen-mapping" "^0.3.0" + jsesc "^2.5.1" + +"@babel/helper-annotate-as-pure@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.16.7.tgz#bb2339a7534a9c128e3102024c60760a3a7f3862" + integrity sha512-s6t2w/IPQVTAET1HitoowRGXooX8mCgtuP5195wD/QJPV6wYjpujCGF7JuMODVX2ZAJOf1GT6DT9MHEZvLOFSw== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-builder-binary-assignment-operator-visitor@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.16.7.tgz#38d138561ea207f0f69eb1626a418e4f7e6a580b" + integrity sha512-C6FdbRaxYjwVu/geKW4ZeQ0Q31AftgRcdSnZ5/jsH6BzCJbtvXvhpfkbkThYSuutZA7nCXpPR6AD9zd1dprMkA== + dependencies: + "@babel/helper-explode-assignable-expression" "^7.16.7" + "@babel/types" "^7.16.7" + +"@babel/helper-compilation-targets@^7.13.0", "@babel/helper-compilation-targets@^7.16.7", "@babel/helper-compilation-targets@^7.17.10", "@babel/helper-compilation-targets@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.18.2.tgz#67a85a10cbd5fc7f1457fec2e7f45441dc6c754b" + integrity sha512-s1jnPotJS9uQnzFtiZVBUxe67CuBa679oWFHpxYYnTpRL/1ffhyX44R9uYiXoa/pLXcY9H2moJta0iaanlk/rQ== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-validator-option" "^7.16.7" + browserslist "^4.20.2" + semver "^6.3.0" + +"@babel/helper-create-class-features-plugin@^7.17.12", "@babel/helper-create-class-features-plugin@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.18.0.tgz#fac430912606331cb075ea8d82f9a4c145a4da19" + integrity sha512-Kh8zTGR9de3J63e5nS0rQUdRs/kbtwoeQQ0sriS0lItjC96u8XXZN6lKpuyWd2coKSU13py/y+LTmThLuVX0Pg== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-environment-visitor" "^7.16.7" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-member-expression-to-functions" "^7.17.7" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/helper-replace-supers" "^7.16.7" + "@babel/helper-split-export-declaration" "^7.16.7" + +"@babel/helper-create-regexp-features-plugin@^7.16.7", "@babel/helper-create-regexp-features-plugin@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.17.12.tgz#bb37ca467f9694bbe55b884ae7a5cc1e0084e4fd" + integrity sha512-b2aZrV4zvutr9AIa6/gA3wsZKRwTKYoDxYiFKcESS3Ug2GTXzwBEvMuuFLhCQpEnRXs1zng4ISAXSUxxKBIcxw== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + regexpu-core "^5.0.1" + +"@babel/helper-define-polyfill-provider@^0.3.1": + version "0.3.1" + resolved "https://registry.yarnpkg.com/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.3.1.tgz#52411b445bdb2e676869e5a74960d2d3826d2665" + integrity sha512-J9hGMpJQmtWmj46B3kBHmL38UhJGhYX7eqkcq+2gsstyYt341HmPeWspihX43yVRA0mS+8GGk2Gckc7bY/HCmA== + dependencies: + "@babel/helper-compilation-targets" "^7.13.0" + "@babel/helper-module-imports" "^7.12.13" + "@babel/helper-plugin-utils" "^7.13.0" + "@babel/traverse" "^7.13.0" + debug "^4.1.1" + lodash.debounce "^4.0.8" + resolve "^1.14.2" + semver "^6.1.2" + +"@babel/helper-environment-visitor@^7.16.7", "@babel/helper-environment-visitor@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.2.tgz#8a6d2dedb53f6bf248e31b4baf38739ee4a637bd" + integrity sha512-14GQKWkX9oJzPiQQ7/J36FTXcD4kSp8egKjO9nINlSKiHITRA9q/R74qu8S9xlc/b/yjsJItQUeeh3xnGN0voQ== + +"@babel/helper-explode-assignable-expression@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-explode-assignable-expression/-/helper-explode-assignable-expression-7.16.7.tgz#12a6d8522fdd834f194e868af6354e8650242b7a" + integrity sha512-KyUenhWMC8VrxzkGP0Jizjo4/Zx+1nNZhgocs+gLzyZyB8SHidhoq9KK/8Ato4anhwsivfkBLftky7gvzbZMtQ== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-function-name@^7.16.7", "@babel/helper-function-name@^7.17.9": + version "7.17.9" + resolved "https://registry.yarnpkg.com/@babel/helper-function-name/-/helper-function-name-7.17.9.tgz#136fcd54bc1da82fcb47565cf16fd8e444b1ff12" + integrity sha512-7cRisGlVtiVqZ0MW0/yFB4atgpGLWEHUVYnb448hZK4x+vih0YO5UoS11XIYtZYqHd0dIPMdUSv8q5K4LdMnIg== + dependencies: + "@babel/template" "^7.16.7" + "@babel/types" "^7.17.0" + +"@babel/helper-hoist-variables@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.16.7.tgz#86bcb19a77a509c7b77d0e22323ef588fa58c246" + integrity sha512-m04d/0Op34H5v7pbZw6pSKP7weA6lsMvfiIAMeIvkY/R4xQtBSMFEigu9QTZ2qB/9l22vsxtM8a+Q8CzD255fg== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-member-expression-to-functions@^7.17.7": + version "7.17.7" + resolved "https://registry.yarnpkg.com/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.17.7.tgz#a34013b57d8542a8c4ff8ba3f747c02452a4d8c4" + integrity sha512-thxXgnQ8qQ11W2wVUObIqDL4p148VMxkt5T/qpN5k2fboRyzFGFmKsTGViquyM5QHKUy48OZoca8kw4ajaDPyw== + dependencies: + "@babel/types" "^7.17.0" + +"@babel/helper-module-imports@^7.12.13", "@babel/helper-module-imports@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.16.7.tgz#25612a8091a999704461c8a222d0efec5d091437" + integrity sha512-LVtS6TqjJHFc+nYeITRo6VLXve70xmq7wPhWTqDJusJEgGmkAACWwMiTNrvfoQo6hEhFwAIixNkvB0jPXDL8Wg== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-module-transforms@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.18.0.tgz#baf05dec7a5875fb9235bd34ca18bad4e21221cd" + integrity sha512-kclUYSUBIjlvnzN2++K9f2qzYKFgjmnmjwL4zlmU5f8ZtzgWe8s0rUPSTGy2HmK4P8T52MQsS+HTQAgZd3dMEA== + dependencies: + "@babel/helper-environment-visitor" "^7.16.7" + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-simple-access" "^7.17.7" + "@babel/helper-split-export-declaration" "^7.16.7" + "@babel/helper-validator-identifier" "^7.16.7" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.0" + "@babel/types" "^7.18.0" + +"@babel/helper-optimise-call-expression@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.16.7.tgz#a34e3560605abbd31a18546bd2aad3e6d9a174f2" + integrity sha512-EtgBhg7rd/JcnpZFXpBy0ze1YRfdm7BnBX4uKMBd3ixa3RGAE002JZB66FJyNH7g0F38U05pXmA5P8cBh7z+1w== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-plugin-utils@^7.0.0", "@babel/helper-plugin-utils@^7.10.4", "@babel/helper-plugin-utils@^7.12.13", "@babel/helper-plugin-utils@^7.13.0", "@babel/helper-plugin-utils@^7.14.5", "@babel/helper-plugin-utils@^7.16.7", "@babel/helper-plugin-utils@^7.17.12", "@babel/helper-plugin-utils@^7.8.0", "@babel/helper-plugin-utils@^7.8.3": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.17.12.tgz#86c2347da5acbf5583ba0a10aed4c9bf9da9cf96" + integrity sha512-JDkf04mqtN3y4iAbO1hv9U2ARpPyPL1zqyWs/2WG1pgSq9llHFjStX5jdxb84himgJm+8Ng+x0oiWF/nw/XQKA== + +"@babel/helper-remap-async-to-generator@^7.16.8": + version "7.16.8" + resolved "https://registry.yarnpkg.com/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.16.8.tgz#29ffaade68a367e2ed09c90901986918d25e57e3" + integrity sha512-fm0gH7Flb8H51LqJHy3HJ3wnE1+qtYR2A99K06ahwrawLdOFsCEWjZOrYricXJHoPSudNKxrMBUPEIPxiIIvBw== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-wrap-function" "^7.16.8" + "@babel/types" "^7.16.8" + +"@babel/helper-replace-supers@^7.16.7", "@babel/helper-replace-supers@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/helper-replace-supers/-/helper-replace-supers-7.18.2.tgz#41fdfcc9abaf900e18ba6e5931816d9062a7b2e0" + integrity sha512-XzAIyxx+vFnrOxiQrToSUOzUOn0e1J2Li40ntddek1Y69AXUTXoDJ40/D5RdjFu7s7qHiaeoTiempZcbuVXh2Q== + dependencies: + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-member-expression-to-functions" "^7.17.7" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/traverse" "^7.18.2" + "@babel/types" "^7.18.2" + +"@babel/helper-simple-access@^7.17.7", "@babel/helper-simple-access@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.18.2.tgz#4dc473c2169ac3a1c9f4a51cfcd091d1c36fcff9" + integrity sha512-7LIrjYzndorDY88MycupkpQLKS1AFfsVRm2k/9PtKScSy5tZq0McZTj+DiMRynboZfIqOKvo03pmhTaUgiD6fQ== + dependencies: + "@babel/types" "^7.18.2" + +"@babel/helper-skip-transparent-expression-wrappers@^7.16.0": + version "7.16.0" + resolved "https://registry.yarnpkg.com/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.16.0.tgz#0ee3388070147c3ae051e487eca3ebb0e2e8bb09" + integrity sha512-+il1gTy0oHwUsBQZyJvukbB4vPMdcYBrFHa0Uc4AizLxbq6BOYC51Rv4tWocX9BLBDLZ4kc6qUFpQ6HRgL+3zw== + dependencies: + "@babel/types" "^7.16.0" + +"@babel/helper-split-export-declaration@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.7.tgz#0b648c0c42da9d3920d85ad585f2778620b8726b" + integrity sha512-xbWoy/PFoxSWazIToT9Sif+jJTlrMcndIsaOKvTA6u7QEo7ilkRZpjew18/W3c7nm8fXdUDXh02VXTbZ0pGDNw== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-validator-identifier@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.16.7.tgz#e8c602438c4a8195751243da9031d1607d247cad" + integrity sha512-hsEnFemeiW4D08A5gUAZxLBTXpZ39P+a+DGDsHw1yxqyQ/jzFEnxf5uTEGp+3bzAbNOxU1paTgYS4ECU/IgfDw== + +"@babel/helper-validator-option@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.16.7.tgz#b203ce62ce5fe153899b617c08957de860de4d23" + integrity sha512-TRtenOuRUVo9oIQGPC5G9DgK4743cdxvtOw0weQNpZXaS16SCBi5MNjZF8vba3ETURjZpTbVn7Vvcf2eAwFozQ== + +"@babel/helper-wrap-function@^7.16.8": + version "7.16.8" + resolved "https://registry.yarnpkg.com/@babel/helper-wrap-function/-/helper-wrap-function-7.16.8.tgz#58afda087c4cd235de92f7ceedebca2c41274200" + integrity sha512-8RpyRVIAW1RcDDGTA+GpPAwV22wXCfKOoM9bet6TLkGIFTkRQSkH1nMQ5Yet4MpoXe1ZwHPVtNasc2w0uZMqnw== + dependencies: + "@babel/helper-function-name" "^7.16.7" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.16.8" + "@babel/types" "^7.16.8" + +"@babel/helpers@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.18.2.tgz#970d74f0deadc3f5a938bfa250738eb4ac889384" + integrity sha512-j+d+u5xT5utcQSzrh9p+PaJX94h++KN+ng9b9WEJq7pkUPAd61FGqhjuUEdfknb3E/uDBb7ruwEeKkIxNJPIrg== + dependencies: + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.2" + "@babel/types" "^7.18.2" + +"@babel/highlight@^7.16.7": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.17.12.tgz#257de56ee5afbd20451ac0a75686b6b404257351" + integrity sha512-7yykMVF3hfZY2jsHZEEgLc+3x4o1O+fYyULu11GynEUQNwB6lua+IIQn1FiJxNucd5UlyJryrwsOh8PL9Sn8Qg== + dependencies: + "@babel/helper-validator-identifier" "^7.16.7" + chalk "^2.0.0" + js-tokens "^4.0.0" + +"@babel/parser@^7.1.0", "@babel/parser@^7.14.7", "@babel/parser@^7.16.7", "@babel/parser@^7.18.0": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.18.4.tgz#6774231779dd700e0af29f6ad8d479582d7ce5ef" + integrity sha512-FDge0dFazETFcxGw/EXzOkN8uJp0PC7Qbm+Pe9T+av2zlBpOgunFHkQPPn+eRuClU73JF+98D531UgayY89tow== + +"@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.17.12.tgz#1dca338caaefca368639c9ffb095afbd4d420b1e" + integrity sha512-xCJQXl4EeQ3J9C4yOmpTrtVGmzpm2iSzyxbkZHw7UCnZBftHpF/hpII80uWVyVrc40ytIClHjgWGTG1g/yB+aw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.17.12.tgz#0d498ec8f0374b1e2eb54b9cb2c4c78714c77753" + integrity sha512-/vt0hpIw0x4b6BLKUkwlvEoiGZYYLNZ96CzyHYPbtG2jZGz6LBe7/V+drYrc/d+ovrF9NBi0pmtvmNb/FsWtRQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + "@babel/plugin-proposal-optional-chaining" "^7.17.12" + +"@babel/plugin-proposal-async-generator-functions@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-async-generator-functions/-/plugin-proposal-async-generator-functions-7.17.12.tgz#094a417e31ce7e692d84bab06c8e2a607cbeef03" + integrity sha512-RWVvqD1ooLKP6IqWTA5GyFVX2isGEgC5iFxKzfYOIy/QEFdxYyCybBDtIGjipHpb9bDWHzcqGqFakf+mVmBTdQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-remap-async-to-generator" "^7.16.8" + "@babel/plugin-syntax-async-generators" "^7.8.4" + +"@babel/plugin-proposal-class-properties@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-class-properties/-/plugin-proposal-class-properties-7.17.12.tgz#84f65c0cc247d46f40a6da99aadd6438315d80a4" + integrity sha512-U0mI9q8pW5Q9EaTHFPwSVusPMV/DV9Mm8p7csqROFLtIE9rBF5piLqyrBGigftALrBcsBGu4m38JneAe7ZDLXw== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-proposal-class-static-block@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-class-static-block/-/plugin-proposal-class-static-block-7.18.0.tgz#7d02253156e3c3793bdb9f2faac3a1c05f0ba710" + integrity sha512-t+8LsRMMDE74c6sV7KShIw13sqbqd58tlqNrsWoWBTIMw7SVQ0cZ905wLNS/FBCy/3PyooRHLFFlfrUNyyz5lA== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + +"@babel/plugin-proposal-dynamic-import@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-dynamic-import/-/plugin-proposal-dynamic-import-7.16.7.tgz#c19c897eaa46b27634a00fee9fb7d829158704b2" + integrity sha512-I8SW9Ho3/8DRSdmDdH3gORdyUuYnk1m4cMxUAdu5oy4n3OfN8flDEH+d60iG7dUfi0KkYwSvoalHzzdRzpWHTg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + +"@babel/plugin-proposal-export-namespace-from@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-export-namespace-from/-/plugin-proposal-export-namespace-from-7.17.12.tgz#b22864ccd662db9606edb2287ea5fd1709f05378" + integrity sha512-j7Ye5EWdwoXOpRmo5QmRyHPsDIe6+u70ZYZrd7uz+ebPYFKfRcLcNu3Ro0vOlJ5zuv8rU7xa+GttNiRzX56snQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + +"@babel/plugin-proposal-json-strings@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-json-strings/-/plugin-proposal-json-strings-7.17.12.tgz#f4642951792437233216d8c1af370bb0fbff4664" + integrity sha512-rKJ+rKBoXwLnIn7n6o6fulViHMrOThz99ybH+hKHcOZbnN14VuMnH9fo2eHE69C8pO4uX1Q7t2HYYIDmv8VYkg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-json-strings" "^7.8.3" + +"@babel/plugin-proposal-logical-assignment-operators@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-logical-assignment-operators/-/plugin-proposal-logical-assignment-operators-7.17.12.tgz#c64a1bcb2b0a6d0ed2ff674fd120f90ee4b88a23" + integrity sha512-EqFo2s1Z5yy+JeJu7SFfbIUtToJTVlC61/C7WLKDntSw4Sz6JNAIfL7zQ74VvirxpjB5kz/kIx0gCcb+5OEo2Q== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + +"@babel/plugin-proposal-nullish-coalescing-operator@^7.14.5", "@babel/plugin-proposal-nullish-coalescing-operator@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-nullish-coalescing-operator/-/plugin-proposal-nullish-coalescing-operator-7.17.12.tgz#1e93079bbc2cbc756f6db6a1925157c4a92b94be" + integrity sha512-ws/g3FSGVzv+VH86+QvgtuJL/kR67xaEIF2x0iPqdDfYW6ra6JF3lKVBkWynRLcNtIC1oCTfDRVxmm2mKzy+ag== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + +"@babel/plugin-proposal-numeric-separator@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-numeric-separator/-/plugin-proposal-numeric-separator-7.16.7.tgz#d6b69f4af63fb38b6ca2558442a7fb191236eba9" + integrity sha512-vQgPMknOIgiuVqbokToyXbkY/OmmjAzr/0lhSIbG/KmnzXPGwW/AdhdKpi+O4X/VkWiWjnkKOBiqJrTaC98VKw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + +"@babel/plugin-proposal-object-rest-spread@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-object-rest-spread/-/plugin-proposal-object-rest-spread-7.18.0.tgz#79f2390c892ba2a68ec112eb0d895cfbd11155e8" + integrity sha512-nbTv371eTrFabDfHLElkn9oyf9VG+VKK6WMzhY2o4eHKaG19BToD9947zzGMO6I/Irstx9d8CwX6njPNIAR/yw== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-compilation-targets" "^7.17.10" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-transform-parameters" "^7.17.12" + +"@babel/plugin-proposal-optional-catch-binding@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-optional-catch-binding/-/plugin-proposal-optional-catch-binding-7.16.7.tgz#c623a430674ffc4ab732fd0a0ae7722b67cb74cf" + integrity sha512-eMOH/L4OvWSZAE1VkHbr1vckLG1WUcHGJSLqqQwl2GaUqG6QjddvrOaTUMNYiv77H5IKPMZ9U9P7EaHwvAShfA== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + +"@babel/plugin-proposal-optional-chaining@^7.14.5", "@babel/plugin-proposal-optional-chaining@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-optional-chaining/-/plugin-proposal-optional-chaining-7.17.12.tgz#f96949e9bacace3a9066323a5cf90cfb9de67174" + integrity sha512-7wigcOs/Z4YWlK7xxjkvaIw84vGhDv/P1dFGQap0nHkc8gFKY/r+hXc8Qzf5k1gY7CvGIcHqAnOagVKJJ1wVOQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + +"@babel/plugin-proposal-private-methods@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-private-methods/-/plugin-proposal-private-methods-7.17.12.tgz#c2ca3a80beb7539289938da005ad525a038a819c" + integrity sha512-SllXoxo19HmxhDWm3luPz+cPhtoTSKLJE9PXshsfrOzBqs60QP0r8OaJItrPhAj0d7mZMnNF0Y1UUggCDgMz1A== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-proposal-private-property-in-object@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.17.12.tgz#b02efb7f106d544667d91ae97405a9fd8c93952d" + integrity sha512-/6BtVi57CJfrtDNKfK5b66ydK2J5pXUKBKSPD2G1whamMuEnZWgoOIfO8Vf9F/DoD4izBLD/Au4NMQfruzzykg== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + +"@babel/plugin-proposal-unicode-property-regex@^7.17.12", "@babel/plugin-proposal-unicode-property-regex@^7.4.4": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-unicode-property-regex/-/plugin-proposal-unicode-property-regex-7.17.12.tgz#3dbd7a67bd7f94c8238b394da112d86aaf32ad4d" + integrity sha512-Wb9qLjXf3ZazqXA7IvI7ozqRIXIGPtSo+L5coFmEkhTQK18ao4UDDD0zdTGAarmbLj2urpRwrc6893cu5Bfh0A== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-async-generators@^7.8.4": + version "7.8.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz#a983fb1aeb2ec3f6ed042a210f640e90e786fe0d" + integrity sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-bigint@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-bigint/-/plugin-syntax-bigint-7.8.3.tgz#4c9a6f669f5d0cdf1b90a1671e9a146be5300cea" + integrity sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-class-properties@^7.12.13", "@babel/plugin-syntax-class-properties@^7.8.3": + version "7.12.13" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz#b5c987274c4a3a82b89714796931a6b53544ae10" + integrity sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA== + dependencies: + "@babel/helper-plugin-utils" "^7.12.13" + +"@babel/plugin-syntax-class-static-block@^7.14.5": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz#195df89b146b4b78b3bf897fd7a257c84659d406" + integrity sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-dynamic-import@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz#62bf98b2da3cd21d626154fc96ee5b3cb68eacb3" + integrity sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-export-namespace-from@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz#028964a9ba80dbc094c915c487ad7c4e7a66465a" + integrity sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.3" + +"@babel/plugin-syntax-flow@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-flow/-/plugin-syntax-flow-7.17.12.tgz#23d852902acd19f42923fca9d0f196984d124e73" + integrity sha512-B8QIgBvkIG6G2jgsOHQUist7Sm0EBLDCx8sen072IwqNuzMegZNXrYnSv77cYzA8mLDZAfQYqsLIhimiP1s2HQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-import-assertions@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-assertions/-/plugin-syntax-import-assertions-7.17.12.tgz#58096a92b11b2e4e54b24c6a0cc0e5e607abcedd" + integrity sha512-n/loy2zkq9ZEM8tEOwON9wTQSTNDTDEz6NujPtJGLU7qObzT1N4c4YZZf8E6ATB2AjNQg/Ib2AIpO03EZaCehw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-import-meta@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-meta/-/plugin-syntax-import-meta-7.10.4.tgz#ee601348c370fa334d2207be158777496521fd51" + integrity sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-json-strings@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz#01ca21b668cd8218c9e640cb6dd88c5412b2c96a" + integrity sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-jsx@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.17.12.tgz#834035b45061983a491f60096f61a2e7c5674a47" + integrity sha512-spyY3E3AURfxh/RHtjx5j6hs8am5NbUBGfcZ2vB3uShSpZdQyXSf5rR5Mk76vbtlAZOelyVQ71Fg0x9SG4fsog== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-logical-assignment-operators@^7.10.4", "@babel/plugin-syntax-logical-assignment-operators@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz#ca91ef46303530448b906652bac2e9fe9941f699" + integrity sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-nullish-coalescing-operator@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz#167ed70368886081f74b5c36c65a88c03b66d1a9" + integrity sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-numeric-separator@^7.10.4", "@babel/plugin-syntax-numeric-separator@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz#b9b070b3e33570cd9fd07ba7fa91c0dd37b9af97" + integrity sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-object-rest-spread@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz#60e225edcbd98a640332a2e72dd3e66f1af55871" + integrity sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-catch-binding@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz#6111a265bcfb020eb9efd0fdfd7d26402b9ed6c1" + integrity sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-chaining@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz#4f69c2ab95167e0180cd5336613f8c5788f7d48a" + integrity sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-private-property-in-object@^7.14.5": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz#0dc6671ec0ea22b6e94a1114f857970cd39de1ad" + integrity sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-top-level-await@^7.14.5", "@babel/plugin-syntax-top-level-await@^7.8.3": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz#c1cfdadc35a646240001f06138247b741c34d94c" + integrity sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-typescript@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.17.12.tgz#b54fc3be6de734a56b87508f99d6428b5b605a7b" + integrity sha512-TYY0SXFiO31YXtNg3HtFwNJHjLsAyIIhAhNWkQ5whPPS7HWUFlg9z0Ta4qAQNjQbP1wsSt/oKkmZ/4/WWdMUpw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-arrow-functions@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.17.12.tgz#dddd783b473b1b1537ef46423e3944ff24898c45" + integrity sha512-PHln3CNi/49V+mza4xMwrg+WGYevSF1oaiXaC2EQfdp4HWlSjRsrDXWJiQBKpP7749u6vQ9mcry2uuFOv5CXvA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-async-to-generator@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.17.12.tgz#dbe5511e6b01eee1496c944e35cdfe3f58050832" + integrity sha512-J8dbrWIOO3orDzir57NRsjg4uxucvhby0L/KZuGsWDj0g7twWK3g7JhJhOrXtuXiw8MeiSdJ3E0OW9H8LYEzLQ== + dependencies: + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-remap-async-to-generator" "^7.16.8" + +"@babel/plugin-transform-block-scoped-functions@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.16.7.tgz#4d0d57d9632ef6062cdf354bb717102ee042a620" + integrity sha512-JUuzlzmF40Z9cXyytcbZEZKckgrQzChbQJw/5PuEHYeqzCsvebDx0K0jWnIIVcmmDOAVctCgnYs0pMcrYj2zJg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-block-scoping@^7.17.12": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.18.4.tgz#7988627b3e9186a13e4d7735dc9c34a056613fb9" + integrity sha512-+Hq10ye+jlvLEogSOtq4mKvtk7qwcUQ1f0Mrueai866C82f844Yom2cttfJdMdqRLTxWpsbfbkIkOIfovyUQXw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-classes@^7.17.12": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-classes/-/plugin-transform-classes-7.18.4.tgz#51310b812a090b846c784e47087fa6457baef814" + integrity sha512-e42NSG2mlKWgxKUAD9EJJSkZxR67+wZqzNxLSpc51T8tRU5SLFHsPmgYR5yr7sdgX4u+iHA1C5VafJ6AyImV3A== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-replace-supers" "^7.18.2" + "@babel/helper-split-export-declaration" "^7.16.7" + globals "^11.1.0" + +"@babel/plugin-transform-computed-properties@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.17.12.tgz#bca616a83679698f3258e892ed422546e531387f" + integrity sha512-a7XINeplB5cQUWMg1E/GI1tFz3LfK021IjV1rj1ypE+R7jHm+pIHmHl25VNkZxtx9uuYp7ThGk8fur1HHG7PgQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-destructuring@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.18.0.tgz#dc4f92587e291b4daa78aa20cc2d7a63aa11e858" + integrity sha512-Mo69klS79z6KEfrLg/1WkmVnB8javh75HX4pi2btjvlIoasuxilEyjtsQW6XPrubNd7AQy0MMaNIaQE4e7+PQw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-dotall-regex@^7.16.7", "@babel/plugin-transform-dotall-regex@^7.4.4": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.16.7.tgz#6b2d67686fab15fb6a7fd4bd895d5982cfc81241" + integrity sha512-Lyttaao2SjZF6Pf4vk1dVKv8YypMpomAbygW+mU5cYP3S5cWTfCJjG8xV6CFdzGFlfWK81IjL9viiTvpb6G7gQ== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-duplicate-keys@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.17.12.tgz#a09aa709a3310013f8e48e0e23bc7ace0f21477c" + integrity sha512-EA5eYFUG6xeerdabina/xIoB95jJ17mAkR8ivx6ZSu9frKShBjpOGZPn511MTDTkiCO+zXnzNczvUM69YSf3Zw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-exponentiation-operator@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.16.7.tgz#efa9862ef97e9e9e5f653f6ddc7b665e8536fe9b" + integrity sha512-8UYLSlyLgRixQvlYH3J2ekXFHDFLQutdy7FfFAMm3CPZ6q9wHCwnUyiXpQCe3gVVnQlHc5nsuiEVziteRNTXEA== + dependencies: + "@babel/helper-builder-binary-assignment-operator-visitor" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-flow-strip-types@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-flow-strip-types/-/plugin-transform-flow-strip-types-7.17.12.tgz#5e070f99a4152194bd9275de140e83a92966cab3" + integrity sha512-g8cSNt+cHCpG/uunPQELdq/TeV3eg1OLJYwxypwHtAWo9+nErH3lQx9CSO2uI9lF74A0mR0t4KoMjs1snSgnTw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-flow" "^7.17.12" + +"@babel/plugin-transform-for-of@^7.18.1": + version "7.18.1" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.18.1.tgz#ed14b657e162b72afbbb2b4cdad277bf2bb32036" + integrity sha512-+TTB5XwvJ5hZbO8xvl2H4XaMDOAK57zF4miuC9qQJgysPNEAZZ9Z69rdF5LJkozGdZrjBIUAIyKUWRMmebI7vg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-function-name@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.16.7.tgz#5ab34375c64d61d083d7d2f05c38d90b97ec65cf" + integrity sha512-SU/C68YVwTRxqWj5kgsbKINakGag0KTgq9f2iZEXdStoAbOzLHEBRYzImmA6yFo8YZhJVflvXmIHUO7GWHmxxA== + dependencies: + "@babel/helper-compilation-targets" "^7.16.7" + "@babel/helper-function-name" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-literals@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-literals/-/plugin-transform-literals-7.17.12.tgz#97131fbc6bbb261487105b4b3edbf9ebf9c830ae" + integrity sha512-8iRkvaTjJciWycPIZ9k9duu663FT7VrBdNqNgxnVXEFwOIp55JWcZd23VBRySYbnS3PwQ3rGiabJBBBGj5APmQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-member-expression-literals@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.16.7.tgz#6e5dcf906ef8a098e630149d14c867dd28f92384" + integrity sha512-mBruRMbktKQwbxaJof32LT9KLy2f3gH+27a5XSuXo6h7R3vqltl0PgZ80C8ZMKw98Bf8bqt6BEVi3svOh2PzMw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-modules-amd@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.18.0.tgz#7ef1002e67e36da3155edc8bf1ac9398064c02ed" + integrity sha512-h8FjOlYmdZwl7Xm2Ug4iX2j7Qy63NANI+NQVWQzv6r25fqgg7k2dZl03p95kvqNclglHs4FZ+isv4p1uXMA+QA== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-commonjs@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.18.2.tgz#1aa8efa2e2a6e818b6a7f2235fceaf09bdb31e9e" + integrity sha512-f5A865gFPAJAEE0K7F/+nm5CmAE3y8AWlMBG9unu5j9+tk50UQVK0QS8RNxSp7MJf0wh97uYyLWt3Zvu71zyOQ== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-simple-access" "^7.18.2" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-systemjs@^7.18.0": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.18.4.tgz#3d6fd9868c735cce8f38d6ae3a407fb7e61e6d46" + integrity sha512-lH2UaQaHVOAeYrUUuZ8i38o76J/FnO8vu21OE+tD1MyP9lxdZoSfz+pDbWkq46GogUrdrMz3tiz/FYGB+bVThg== + dependencies: + "@babel/helper-hoist-variables" "^7.16.7" + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-identifier" "^7.16.7" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-umd@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.18.0.tgz#56aac64a2c2a1922341129a4597d1fd5c3ff020f" + integrity sha512-d/zZ8I3BWli1tmROLxXLc9A6YXvGK8egMxHp+E/rRwMh1Kip0AP77VwZae3snEJ33iiWwvNv2+UIIhfalqhzZA== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-named-capturing-groups-regex@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.17.12.tgz#9c4a5a5966e0434d515f2675c227fd8cc8606931" + integrity sha512-vWoWFM5CKaTeHrdUJ/3SIOTRV+MBVGybOC9mhJkaprGNt5demMymDW24yC74avb915/mIRe3TgNb/d8idvnCRA== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-new-target@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.17.12.tgz#10842cd605a620944e81ea6060e9e65c265742e3" + integrity sha512-CaOtzk2fDYisbjAD4Sd1MTKGVIpRtx9bWLyj24Y/k6p4s4gQ3CqDGJauFJxt8M/LEx003d0i3klVqnN73qvK3w== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-object-super@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.16.7.tgz#ac359cf8d32cf4354d27a46867999490b6c32a94" + integrity sha512-14J1feiQVWaGvRxj2WjyMuXS2jsBkgB3MdSN5HuC2G5nRspa5RK9COcs82Pwy5BuGcjb+fYaUj94mYcOj7rCvw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/helper-replace-supers" "^7.16.7" + +"@babel/plugin-transform-parameters@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.17.12.tgz#eb467cd9586ff5ff115a9880d6fdbd4a846b7766" + integrity sha512-6qW4rWo1cyCdq1FkYri7AHpauchbGLXpdwnYsfxFb+KtddHENfsY5JZb35xUwkK5opOLcJ3BNd2l7PhRYGlwIA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-property-literals@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.16.7.tgz#2dadac85155436f22c696c4827730e0fe1057a55" + integrity sha512-z4FGr9NMGdoIl1RqavCqGG+ZuYjfZ/hkCIeuH6Do7tXmSm0ls11nYVSJqFEUOSJbDab5wC6lRE/w6YjVcr6Hqw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-react-display-name@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.16.7.tgz#7b6d40d232f4c0f550ea348593db3b21e2404340" + integrity sha512-qgIg8BcZgd0G/Cz916D5+9kqX0c7nPZyXaP8R2tLNN5tkyIZdG5fEwBrxwplzSnjC1jvQmyMNVwUCZPcbGY7Pg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-react-jsx-development@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.16.7.tgz#43a00724a3ed2557ed3f276a01a929e6686ac7b8" + integrity sha512-RMvQWvpla+xy6MlBpPlrKZCMRs2AGiHOGHY3xRwl0pEeim348dDyxeH4xBsMPbIMhujeq7ihE702eM2Ew0Wo+A== + dependencies: + "@babel/plugin-transform-react-jsx" "^7.16.7" + +"@babel/plugin-transform-react-jsx@^7.16.7", "@babel/plugin-transform-react-jsx@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.17.12.tgz#2aa20022709cd6a3f40b45d60603d5f269586dba" + integrity sha512-Lcaw8bxd1DKht3thfD4A12dqo1X16he1Lm8rIv8sTwjAYNInRS1qHa9aJoqvzpscItXvftKDCfaEQzwoVyXpEQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-jsx" "^7.17.12" + "@babel/types" "^7.17.12" + +"@babel/plugin-transform-react-pure-annotations@^7.16.7": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.18.0.tgz#ef82c8e310913f3522462c9ac967d395092f1954" + integrity sha512-6+0IK6ouvqDn9bmEG7mEyF/pwlJXVj5lwydybpyyH3D0A7Hftk+NCTdYjnLNZksn261xaOV5ksmp20pQEmc2RQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-regenerator@^7.18.0": + version "7.18.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.18.0.tgz#44274d655eb3f1af3f3a574ba819d3f48caf99d5" + integrity sha512-C8YdRw9uzx25HSIzwA7EM7YP0FhCe5wNvJbZzjVNHHPGVcDJ3Aie+qGYYdS1oVQgn+B3eAIJbWFLrJ4Jipv7nw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + regenerator-transform "^0.15.0" + +"@babel/plugin-transform-reserved-words@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.17.12.tgz#7dbd349f3cdffba751e817cf40ca1386732f652f" + integrity sha512-1KYqwbJV3Co03NIi14uEHW8P50Md6KqFgt0FfpHdK6oyAHQVTosgPuPSiWud1HX0oYJ1hGRRlk0fP87jFpqXZA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-runtime@^7.15.0": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.18.2.tgz#04637de1e45ae8847ff14b9beead09c33d34374d" + integrity sha512-mr1ufuRMfS52ttq+1G1PD8OJNqgcTFjq3hwn8SZ5n1x1pBhi0E36rYMdTK0TsKtApJ4lDEdfXJwtGobQMHSMPg== + dependencies: + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + babel-plugin-polyfill-corejs2 "^0.3.0" + babel-plugin-polyfill-corejs3 "^0.5.0" + babel-plugin-polyfill-regenerator "^0.3.0" + semver "^6.3.0" + +"@babel/plugin-transform-shorthand-properties@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.16.7.tgz#e8549ae4afcf8382f711794c0c7b6b934c5fbd2a" + integrity sha512-hah2+FEnoRoATdIb05IOXf+4GzXYTq75TVhIn1PewihbpyrNWUt2JbudKQOETWw6QpLe+AIUpJ5MVLYTQbeeUg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-spread@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-spread/-/plugin-transform-spread-7.17.12.tgz#c112cad3064299f03ea32afed1d659223935d1f5" + integrity sha512-9pgmuQAtFi3lpNUstvG9nGfk9DkrdmWNp9KeKPFmuZCpEnxRzYlS8JgwPjYj+1AWDOSvoGN0H30p1cBOmT/Svg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + +"@babel/plugin-transform-sticky-regex@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.16.7.tgz#c84741d4f4a38072b9a1e2e3fd56d359552e8660" + integrity sha512-NJa0Bd/87QV5NZZzTuZG5BPJjLYadeSZ9fO6oOUoL4iQx+9EEuw/eEM92SrsT19Yc2jgB1u1hsjqDtH02c3Drw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-template-literals@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.18.2.tgz#31ed6915721864847c48b656281d0098ea1add28" + integrity sha512-/cmuBVw9sZBGZVOMkpAEaVLwm4JmK2GZ1dFKOGGpMzEHWFmyZZ59lUU0PdRr8YNYeQdNzTDwuxP2X2gzydTc9g== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-typeof-symbol@^7.17.12": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.17.12.tgz#0f12f57ac35e98b35b4ed34829948d42bd0e6889" + integrity sha512-Q8y+Jp7ZdtSPXCThB6zjQ74N3lj0f6TDh1Hnf5B+sYlzQ8i5Pjp8gW0My79iekSpT4WnI06blqP6DT0OmaXXmw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-typescript@^7.15.4": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.18.4.tgz#587eaf6a39edb8c06215e550dc939faeadd750bf" + integrity sha512-l4vHuSLUajptpHNEOUDEGsnpl9pfRLsN1XUoDQDD/YBuXTM+v37SHGS+c6n4jdcZy96QtuUuSvZYMLSSsjH8Mw== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-typescript" "^7.17.12" + +"@babel/plugin-transform-unicode-escapes@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.16.7.tgz#da8717de7b3287a2c6d659750c964f302b31ece3" + integrity sha512-TAV5IGahIz3yZ9/Hfv35TV2xEm+kaBDaZQCn2S/hG9/CZ0DktxJv9eKfPc7yYCvOYR4JGx1h8C+jcSOvgaaI/Q== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-unicode-regex@^7.16.7": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.16.7.tgz#0f7aa4a501198976e25e82702574c34cfebe9ef2" + integrity sha512-oC5tYYKw56HO75KZVLQ+R/Nl3Hro9kf8iG0hXoaHP7tjAyCpvqBiSNe6vGrZni1Z6MggmUOC6A7VP7AVmw225Q== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/preset-env@^7.15.4": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/preset-env/-/preset-env-7.18.2.tgz#f47d3000a098617926e674c945d95a28cb90977a" + integrity sha512-PfpdxotV6afmXMU47S08F9ZKIm2bJIQ0YbAAtDfIENX7G1NUAXigLREh69CWDjtgUy7dYn7bsMzkgdtAlmS68Q== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-compilation-targets" "^7.18.2" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression" "^7.17.12" + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining" "^7.17.12" + "@babel/plugin-proposal-async-generator-functions" "^7.17.12" + "@babel/plugin-proposal-class-properties" "^7.17.12" + "@babel/plugin-proposal-class-static-block" "^7.18.0" + "@babel/plugin-proposal-dynamic-import" "^7.16.7" + "@babel/plugin-proposal-export-namespace-from" "^7.17.12" + "@babel/plugin-proposal-json-strings" "^7.17.12" + "@babel/plugin-proposal-logical-assignment-operators" "^7.17.12" + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.17.12" + "@babel/plugin-proposal-numeric-separator" "^7.16.7" + "@babel/plugin-proposal-object-rest-spread" "^7.18.0" + "@babel/plugin-proposal-optional-catch-binding" "^7.16.7" + "@babel/plugin-proposal-optional-chaining" "^7.17.12" + "@babel/plugin-proposal-private-methods" "^7.17.12" + "@babel/plugin-proposal-private-property-in-object" "^7.17.12" + "@babel/plugin-proposal-unicode-property-regex" "^7.17.12" + "@babel/plugin-syntax-async-generators" "^7.8.4" + "@babel/plugin-syntax-class-properties" "^7.12.13" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + "@babel/plugin-syntax-import-assertions" "^7.17.12" + "@babel/plugin-syntax-json-strings" "^7.8.3" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + "@babel/plugin-syntax-top-level-await" "^7.14.5" + "@babel/plugin-transform-arrow-functions" "^7.17.12" + "@babel/plugin-transform-async-to-generator" "^7.17.12" + "@babel/plugin-transform-block-scoped-functions" "^7.16.7" + "@babel/plugin-transform-block-scoping" "^7.17.12" + "@babel/plugin-transform-classes" "^7.17.12" + "@babel/plugin-transform-computed-properties" "^7.17.12" + "@babel/plugin-transform-destructuring" "^7.18.0" + "@babel/plugin-transform-dotall-regex" "^7.16.7" + "@babel/plugin-transform-duplicate-keys" "^7.17.12" + "@babel/plugin-transform-exponentiation-operator" "^7.16.7" + "@babel/plugin-transform-for-of" "^7.18.1" + "@babel/plugin-transform-function-name" "^7.16.7" + "@babel/plugin-transform-literals" "^7.17.12" + "@babel/plugin-transform-member-expression-literals" "^7.16.7" + "@babel/plugin-transform-modules-amd" "^7.18.0" + "@babel/plugin-transform-modules-commonjs" "^7.18.2" + "@babel/plugin-transform-modules-systemjs" "^7.18.0" + "@babel/plugin-transform-modules-umd" "^7.18.0" + "@babel/plugin-transform-named-capturing-groups-regex" "^7.17.12" + "@babel/plugin-transform-new-target" "^7.17.12" + "@babel/plugin-transform-object-super" "^7.16.7" + "@babel/plugin-transform-parameters" "^7.17.12" + "@babel/plugin-transform-property-literals" "^7.16.7" + "@babel/plugin-transform-regenerator" "^7.18.0" + "@babel/plugin-transform-reserved-words" "^7.17.12" + "@babel/plugin-transform-shorthand-properties" "^7.16.7" + "@babel/plugin-transform-spread" "^7.17.12" + "@babel/plugin-transform-sticky-regex" "^7.16.7" + "@babel/plugin-transform-template-literals" "^7.18.2" + "@babel/plugin-transform-typeof-symbol" "^7.17.12" + "@babel/plugin-transform-unicode-escapes" "^7.16.7" + "@babel/plugin-transform-unicode-regex" "^7.16.7" + "@babel/preset-modules" "^0.1.5" + "@babel/types" "^7.18.2" + babel-plugin-polyfill-corejs2 "^0.3.0" + babel-plugin-polyfill-corejs3 "^0.5.0" + babel-plugin-polyfill-regenerator "^0.3.0" + core-js-compat "^3.22.1" + semver "^6.3.0" + +"@babel/preset-flow@^7.14.0": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/preset-flow/-/preset-flow-7.17.12.tgz#664a5df59190260939eee862800a255bef3bd66f" + integrity sha512-7QDz7k4uiaBdu7N89VKjUn807pJRXmdirQu0KyR9LXnQrr5Jt41eIMKTS7ljej+H29erwmMrwq9Io9mJHLI3Lw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-transform-flow-strip-types" "^7.17.12" + +"@babel/preset-modules@^0.1.5": + version "0.1.5" + resolved "https://registry.yarnpkg.com/@babel/preset-modules/-/preset-modules-0.1.5.tgz#ef939d6e7f268827e1841638dc6ff95515e115d9" + integrity sha512-A57th6YRG7oR3cq/yt/Y84MvGgE0eJG2F1JLhKuyG+jFxEgrd/HAMJatiFtmOiZurz+0DkrvbheCLaV5f2JfjA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@babel/plugin-proposal-unicode-property-regex" "^7.4.4" + "@babel/plugin-transform-dotall-regex" "^7.4.4" + "@babel/types" "^7.4.4" + esutils "^2.0.2" + +"@babel/preset-react@^7.14.0": + version "7.17.12" + resolved "https://registry.yarnpkg.com/@babel/preset-react/-/preset-react-7.17.12.tgz#62adbd2d1870c0de3893095757ed5b00b492ab3d" + integrity sha512-h5U+rwreXtZaRBEQhW1hOJLMq8XNJBQ/9oymXiCXTuT/0uOwpbT0gUt+sXeOqoXBgNuUKI7TaObVwoEyWkpFgA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-transform-react-display-name" "^7.16.7" + "@babel/plugin-transform-react-jsx" "^7.17.12" + "@babel/plugin-transform-react-jsx-development" "^7.16.7" + "@babel/plugin-transform-react-pure-annotations" "^7.16.7" + +"@babel/runtime@^7.13.10", "@babel/runtime@^7.15.4", "@babel/runtime@^7.8.4": + version "7.18.3" + resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.18.3.tgz#c7b654b57f6f63cf7f8b418ac9ca04408c4579f4" + integrity sha512-38Y8f7YUhce/K7RMwTp7m0uCumpv9hZkitCbBClqQIow1qSbCvGkcegKOXpEWCQLfWmevgRiWokZ1GkpfhbZug== + dependencies: + regenerator-runtime "^0.13.4" + +"@babel/template@^7.16.7", "@babel/template@^7.3.3": + version "7.16.7" + resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.16.7.tgz#8d126c8701fde4d66b264b3eba3d96f07666d155" + integrity sha512-I8j/x8kHUrbYRTUxXrrMbfCa7jxkE7tZre39x3kjr9hvI82cK1FfqLygotcWN5kdPGWcLdWMHpSBavse5tWw3w== + dependencies: + "@babel/code-frame" "^7.16.7" + "@babel/parser" "^7.16.7" + "@babel/types" "^7.16.7" + +"@babel/traverse@^7.1.0", "@babel/traverse@^7.13.0", "@babel/traverse@^7.16.8", "@babel/traverse@^7.18.0", "@babel/traverse@^7.18.2": + version "7.18.2" + resolved "https://registry.yarnpkg.com/@babel/traverse/-/traverse-7.18.2.tgz#b77a52604b5cc836a9e1e08dca01cba67a12d2e8" + integrity sha512-9eNwoeovJ6KH9zcCNnENY7DMFwTU9JdGCFtqNLfUAqtUHRCOsTOqWoffosP8vKmNYeSBUv3yVJXjfd8ucwOjUA== + dependencies: + "@babel/code-frame" "^7.16.7" + "@babel/generator" "^7.18.2" + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-hoist-variables" "^7.16.7" + "@babel/helper-split-export-declaration" "^7.16.7" + "@babel/parser" "^7.18.0" + "@babel/types" "^7.18.2" + debug "^4.1.0" + globals "^11.1.0" + +"@babel/types@^7.0.0", "@babel/types@^7.16.0", "@babel/types@^7.16.7", "@babel/types@^7.16.8", "@babel/types@^7.17.0", "@babel/types@^7.17.12", "@babel/types@^7.18.0", "@babel/types@^7.18.2", "@babel/types@^7.3.0", "@babel/types@^7.3.3", "@babel/types@^7.4.4": + version "7.18.4" + resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.18.4.tgz#27eae9b9fd18e9dccc3f9d6ad051336f307be354" + integrity sha512-ThN1mBcMq5pG/Vm2IcBmPPfyPXbd8S02rS+OBIDENdufvqC7Z/jHPCv9IcP01277aKtDI8g/2XysBN4hA8niiw== + dependencies: + "@babel/helper-validator-identifier" "^7.16.7" + to-fast-properties "^2.0.0" + +"@bcoe/v8-coverage@^0.2.3": + version "0.2.3" + resolved "https://registry.yarnpkg.com/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz#75a2e8b51cb758a7553d6804a5932d7aace75c39" + integrity sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw== + +"@cnakazawa/watch@^1.0.3": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@cnakazawa/watch/-/watch-1.0.4.tgz#f864ae85004d0fcab6f50be9141c4da368d1656a" + integrity sha512-v9kIhKwjeZThiWrLmj0y17CWoyddASLj9O2yvbZkbvw/N3rWOYy9zkV66ursAoVr0mV15bL8g0c4QZUE6cdDoQ== + dependencies: + exec-sh "^0.3.2" + minimist "^1.2.0" + +"@istanbuljs/load-nyc-config@^1.0.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz#fd3db1d59ecf7cf121e80650bb86712f9b55eced" + integrity sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ== + dependencies: + camelcase "^5.3.1" + find-up "^4.1.0" + get-package-type "^0.1.0" + js-yaml "^3.13.1" + resolve-from "^5.0.0" + +"@istanbuljs/schema@^0.1.2": + version "0.1.3" + resolved "https://registry.yarnpkg.com/@istanbuljs/schema/-/schema-0.1.3.tgz#e45e384e4b8ec16bce2fd903af78450f6bf7ec98" + integrity sha512-ZXRY4jNvVgSVQ8DL3LTcakaAtXwTVUxE81hslsyD2AtoXW/wVob10HkOJ1X/pAlcI7D+2YoZKg5do8G/w6RYgA== + +"@jest/console@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/console/-/console-26.6.2.tgz#4e04bc464014358b03ab4937805ee36a0aeb98f2" + integrity sha512-IY1R2i2aLsLr7Id3S6p2BA82GNWryt4oSvEXLAKc+L2zdi89dSkE8xC1C+0kpATG4JhBJREnQOH7/zmccM2B0g== + dependencies: + "@jest/types" "^26.6.2" + "@types/node" "*" + chalk "^4.0.0" + jest-message-util "^26.6.2" + jest-util "^26.6.2" + slash "^3.0.0" + +"@jest/core@^26.6.3": + version "26.6.3" + resolved "https://registry.yarnpkg.com/@jest/core/-/core-26.6.3.tgz#7639fcb3833d748a4656ada54bde193051e45fad" + integrity sha512-xvV1kKbhfUqFVuZ8Cyo+JPpipAHHAV3kcDBftiduK8EICXmTFddryy3P7NfZt8Pv37rA9nEJBKCCkglCPt/Xjw== + dependencies: + "@jest/console" "^26.6.2" + "@jest/reporters" "^26.6.2" + "@jest/test-result" "^26.6.2" + "@jest/transform" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + ansi-escapes "^4.2.1" + chalk "^4.0.0" + exit "^0.1.2" + graceful-fs "^4.2.4" + jest-changed-files "^26.6.2" + jest-config "^26.6.3" + jest-haste-map "^26.6.2" + jest-message-util "^26.6.2" + jest-regex-util "^26.0.0" + jest-resolve "^26.6.2" + jest-resolve-dependencies "^26.6.3" + jest-runner "^26.6.3" + jest-runtime "^26.6.3" + jest-snapshot "^26.6.2" + jest-util "^26.6.2" + jest-validate "^26.6.2" + jest-watcher "^26.6.2" + micromatch "^4.0.2" + p-each-series "^2.1.0" + rimraf "^3.0.0" + slash "^3.0.0" + strip-ansi "^6.0.0" + +"@jest/environment@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/environment/-/environment-26.6.2.tgz#ba364cc72e221e79cc8f0a99555bf5d7577cf92c" + integrity sha512-nFy+fHl28zUrRsCeMB61VDThV1pVTtlEokBRgqPrcT1JNq4yRNIyTHfyht6PqtUvY9IsuLGTrbG8kPXjSZIZwA== + dependencies: + "@jest/fake-timers" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + jest-mock "^26.6.2" + +"@jest/fake-timers@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/fake-timers/-/fake-timers-26.6.2.tgz#459c329bcf70cee4af4d7e3f3e67848123535aad" + integrity sha512-14Uleatt7jdzefLPYM3KLcnUl1ZNikaKq34enpb5XG9i81JpppDb5muZvonvKyrl7ftEHkKS5L5/eB/kxJ+bvA== + dependencies: + "@jest/types" "^26.6.2" + "@sinonjs/fake-timers" "^6.0.1" + "@types/node" "*" + jest-message-util "^26.6.2" + jest-mock "^26.6.2" + jest-util "^26.6.2" + +"@jest/globals@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/globals/-/globals-26.6.2.tgz#5b613b78a1aa2655ae908eba638cc96a20df720a" + integrity sha512-85Ltnm7HlB/KesBUuALwQ68YTU72w9H2xW9FjZ1eL1U3lhtefjjl5c2MiUbpXt/i6LaPRvoOFJ22yCBSfQ0JIA== + dependencies: + "@jest/environment" "^26.6.2" + "@jest/types" "^26.6.2" + expect "^26.6.2" + +"@jest/reporters@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/reporters/-/reporters-26.6.2.tgz#1f518b99637a5f18307bd3ecf9275f6882a667f6" + integrity sha512-h2bW53APG4HvkOnVMo8q3QXa6pcaNt1HkwVsOPMBV6LD/q9oSpxNSYZQYkAnjdMjrJ86UuYeLo+aEZClV6opnw== + dependencies: + "@bcoe/v8-coverage" "^0.2.3" + "@jest/console" "^26.6.2" + "@jest/test-result" "^26.6.2" + "@jest/transform" "^26.6.2" + "@jest/types" "^26.6.2" + chalk "^4.0.0" + collect-v8-coverage "^1.0.0" + exit "^0.1.2" + glob "^7.1.2" + graceful-fs "^4.2.4" + istanbul-lib-coverage "^3.0.0" + istanbul-lib-instrument "^4.0.3" + istanbul-lib-report "^3.0.0" + istanbul-lib-source-maps "^4.0.0" + istanbul-reports "^3.0.2" + jest-haste-map "^26.6.2" + jest-resolve "^26.6.2" + jest-util "^26.6.2" + jest-worker "^26.6.2" + slash "^3.0.0" + source-map "^0.6.0" + string-length "^4.0.1" + terminal-link "^2.0.0" + v8-to-istanbul "^7.0.0" + optionalDependencies: + node-notifier "^8.0.0" + +"@jest/source-map@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/source-map/-/source-map-26.6.2.tgz#29af5e1e2e324cafccc936f218309f54ab69d535" + integrity sha512-YwYcCwAnNmOVsZ8mr3GfnzdXDAl4LaenZP5z+G0c8bzC9/dugL8zRmxZzdoTl4IaS3CryS1uWnROLPFmb6lVvA== + dependencies: + callsites "^3.0.0" + graceful-fs "^4.2.4" + source-map "^0.6.0" + +"@jest/test-result@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/test-result/-/test-result-26.6.2.tgz#55da58b62df134576cc95476efa5f7949e3f5f18" + integrity sha512-5O7H5c/7YlojphYNrK02LlDIV2GNPYisKwHm2QTKjNZeEzezCbwYs9swJySv2UfPMyZ0VdsmMv7jIlD/IKYQpQ== + dependencies: + "@jest/console" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/istanbul-lib-coverage" "^2.0.0" + collect-v8-coverage "^1.0.0" + +"@jest/test-sequencer@^26.6.3": + version "26.6.3" + resolved "https://registry.yarnpkg.com/@jest/test-sequencer/-/test-sequencer-26.6.3.tgz#98e8a45100863886d074205e8ffdc5a7eb582b17" + integrity sha512-YHlVIjP5nfEyjlrSr8t/YdNfU/1XEt7c5b4OxcXCjyRhjzLYu/rO69/WHPuYcbCWkz8kAeZVZp2N2+IOLLEPGw== + dependencies: + "@jest/test-result" "^26.6.2" + graceful-fs "^4.2.4" + jest-haste-map "^26.6.2" + jest-runner "^26.6.3" + jest-runtime "^26.6.3" + +"@jest/transform@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/transform/-/transform-26.6.2.tgz#5ac57c5fa1ad17b2aae83e73e45813894dcf2e4b" + integrity sha512-E9JjhUgNzvuQ+vVAL21vlyfy12gP0GhazGgJC4h6qUt1jSdUXGWJ1wfu/X7Sd8etSgxV4ovT1pb9v5D6QW4XgA== + dependencies: + "@babel/core" "^7.1.0" + "@jest/types" "^26.6.2" + babel-plugin-istanbul "^6.0.0" + chalk "^4.0.0" + convert-source-map "^1.4.0" + fast-json-stable-stringify "^2.0.0" + graceful-fs "^4.2.4" + jest-haste-map "^26.6.2" + jest-regex-util "^26.0.0" + jest-util "^26.6.2" + micromatch "^4.0.2" + pirates "^4.0.1" + slash "^3.0.0" + source-map "^0.6.1" + write-file-atomic "^3.0.0" + +"@jest/types@^26.6.2": + version "26.6.2" + resolved "https://registry.yarnpkg.com/@jest/types/-/types-26.6.2.tgz#bef5a532030e1d88a2f5a6d933f84e97226ed48e" + integrity sha512-fC6QCp7Sc5sX6g8Tvbmj4XUTbyrik0akgRy03yjXbQaBWWNWGE7SGtJk98m0N8nzegD/7SggrUlivxo5ax4KWQ== + dependencies: + "@types/istanbul-lib-coverage" "^2.0.0" + "@types/istanbul-reports" "^3.0.0" + "@types/node" "*" + "@types/yargs" "^15.0.0" + chalk "^4.0.0" + +"@jridgewell/gen-mapping@^0.1.0": + version "0.1.1" + resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.1.1.tgz#e5d2e450306a9491e3bd77e323e38d7aff315996" + integrity sha512-sQXCasFk+U8lWYEe66WxRDOE9PjVz4vSM51fTu3Hw+ClTpUSQb718772vH3pyS5pShp6lvQM7SxgIDXXXmOX7w== + dependencies: + "@jridgewell/set-array" "^1.0.0" + "@jridgewell/sourcemap-codec" "^1.4.10" + +"@jridgewell/gen-mapping@^0.3.0": + version "0.3.1" + resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.1.tgz#cf92a983c83466b8c0ce9124fadeaf09f7c66ea9" + integrity sha512-GcHwniMlA2z+WFPWuY8lp3fsza0I8xPFMWL5+n8LYyP6PSvPrXf4+n8stDHZY2DM0zy9sVkRDy1jDI4XGzYVqg== + dependencies: + "@jridgewell/set-array" "^1.0.0" + "@jridgewell/sourcemap-codec" "^1.4.10" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/resolve-uri@^3.0.3": + version "3.0.7" + resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.0.7.tgz#30cd49820a962aff48c8fffc5cd760151fca61fe" + integrity sha512-8cXDaBBHOr2pQ7j77Y6Vp5VDT2sIqWyWQ56TjEq4ih/a4iST3dItRe8Q9fp0rrIl9DoKhWQtUQz/YpOxLkXbNA== + +"@jridgewell/set-array@^1.0.0": + version "1.1.1" + resolved "https://registry.yarnpkg.com/@jridgewell/set-array/-/set-array-1.1.1.tgz#36a6acc93987adcf0ba50c66908bd0b70de8afea" + integrity sha512-Ct5MqZkLGEXTVmQYbGtx9SVqD2fqwvdubdps5D3djjAkgkKwT918VNOz65pEHFaYTeWcukmJmH5SwsA9Tn2ObQ== + +"@jridgewell/sourcemap-codec@^1.4.10": + version "1.4.13" + resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.13.tgz#b6461fb0c2964356c469e115f504c95ad97ab88c" + integrity sha512-GryiOJmNcWbovBxTfZSF71V/mXbgcV3MewDe3kIMCLyIh5e7SKAeUZs+rMnJ8jkMolZ/4/VsdBmMrw3l+VdZ3w== + +"@jridgewell/trace-mapping@^0.3.8", "@jridgewell/trace-mapping@^0.3.9": + version "0.3.13" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.13.tgz#dcfe3e95f224c8fe97a87a5235defec999aa92ea" + integrity sha512-o1xbKhp9qnIAoHJSWd6KlCZfqslL4valSF81H8ImioOAxluWYWOpWkpyktY2vnt4tbrX9XYaxovq6cgowaJp2w== + dependencies: + "@jridgewell/resolve-uri" "^3.0.3" + "@jridgewell/sourcemap-codec" "^1.4.10" + +"@nicolo-ribaudo/chokidar-2@2.1.8-no-fsevents.3": + version "2.1.8-no-fsevents.3" + resolved "https://registry.yarnpkg.com/@nicolo-ribaudo/chokidar-2/-/chokidar-2-2.1.8-no-fsevents.3.tgz#323d72dd25103d0c4fbdce89dadf574a787b1f9b" + integrity sha512-s88O1aVtXftvp5bCPB7WnmXc5IwOZZ7YPuwNPt+GtOOXpPvad1LfbmjYv+qII7zP6RU2QGnqve27dnLycEnyEQ== + +"@sinonjs/commons@^1.7.0": + version "1.8.3" + resolved "https://registry.yarnpkg.com/@sinonjs/commons/-/commons-1.8.3.tgz#3802ddd21a50a949b6721ddd72da36e67e7f1b2d" + integrity sha512-xkNcLAn/wZaX14RPlwizcKicDk9G3F8m2nU3L7Ukm5zBgTwiT0wsoFAHx9Jq56fJA1z/7uKGtCRu16sOUCLIHQ== + dependencies: + type-detect "4.0.8" + +"@sinonjs/fake-timers@^6.0.1": + version "6.0.1" + resolved "https://registry.yarnpkg.com/@sinonjs/fake-timers/-/fake-timers-6.0.1.tgz#293674fccb3262ac782c7aadfdeca86b10c75c40" + integrity sha512-MZPUxrmFubI36XS1DI3qmI0YdN1gks62JtFZvxR67ljjSNCeK6U08Zx4msEWOXuofgqUt6zPHSi1H9fbjR/NRA== + dependencies: + "@sinonjs/commons" "^1.7.0" + +"@tootallnate/once@1": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@tootallnate/once/-/once-1.1.2.tgz#ccb91445360179a04e7fe6aff78c00ffc1eeaf82" + integrity sha512-RbzJvlNzmRq5c3O09UipeuXno4tA1FE6ikOjxZK0tuxVv3412l64l5t1W5pj4+rJq9vpkm/kwiR07aZXnsKPxw== + +"@types/babel__core@^7.0.0", "@types/babel__core@^7.1.7": + version "7.1.19" + resolved "https://registry.yarnpkg.com/@types/babel__core/-/babel__core-7.1.19.tgz#7b497495b7d1b4812bdb9d02804d0576f43ee460" + integrity sha512-WEOTgRsbYkvA/KCsDwVEGkd7WAr1e3g31VHQ8zy5gul/V1qKullU/BU5I68X5v7V3GnB9eotmom4v5a5gjxorw== + dependencies: + "@babel/parser" "^7.1.0" + "@babel/types" "^7.0.0" + "@types/babel__generator" "*" + "@types/babel__template" "*" + "@types/babel__traverse" "*" + +"@types/babel__generator@*": + version "7.6.4" + resolved "https://registry.yarnpkg.com/@types/babel__generator/-/babel__generator-7.6.4.tgz#1f20ce4c5b1990b37900b63f050182d28c2439b7" + integrity sha512-tFkciB9j2K755yrTALxD44McOrk+gfpIpvC3sxHjRawj6PfnQxrse4Clq5y/Rq+G3mrBurMax/lG8Qn2t9mSsg== + dependencies: + "@babel/types" "^7.0.0" + +"@types/babel__template@*": + version "7.4.1" + resolved "https://registry.yarnpkg.com/@types/babel__template/-/babel__template-7.4.1.tgz#3d1a48fd9d6c0edfd56f2ff578daed48f36c8969" + integrity sha512-azBFKemX6kMg5Io+/rdGT0dkGreboUVR0Cdm3fz9QJWpaQGJRQXl7C+6hOTCZcMll7KFyEQpgbYI2lHdsS4U7g== + dependencies: + "@babel/parser" "^7.1.0" + "@babel/types" "^7.0.0" + +"@types/babel__traverse@*", "@types/babel__traverse@^7.0.4", "@types/babel__traverse@^7.0.6": + version "7.17.1" + resolved "https://registry.yarnpkg.com/@types/babel__traverse/-/babel__traverse-7.17.1.tgz#1a0e73e8c28c7e832656db372b779bfd2ef37314" + integrity sha512-kVzjari1s2YVi77D3w1yuvohV2idweYXMCDzqBiVNN63TcDWrIlTVOYpqVrvbbyOE/IyzBoTKF0fdnLPEORFxA== + dependencies: + "@babel/types" "^7.3.0" + +"@types/graceful-fs@^4.1.2": + version "4.1.5" + resolved "https://registry.yarnpkg.com/@types/graceful-fs/-/graceful-fs-4.1.5.tgz#21ffba0d98da4350db64891f92a9e5db3cdb4e15" + integrity sha512-anKkLmZZ+xm4p8JWBf4hElkM4XR+EZeA2M9BAkkTldmcyDY4mbdIJnRghDJH3Ov5ooY7/UAoENtmdMSkaAd7Cw== + dependencies: + "@types/node" "*" + +"@types/istanbul-lib-coverage@*", "@types/istanbul-lib-coverage@^2.0.0", "@types/istanbul-lib-coverage@^2.0.1": + version "2.0.4" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.4.tgz#8467d4b3c087805d63580480890791277ce35c44" + integrity sha512-z/QT1XN4K4KYuslS23k62yDIDLwLFkzxOuMplDtObz0+y7VqJCaO2o+SPwHCvLFZh7xazvvoor2tA/hPz9ee7g== + +"@types/istanbul-lib-report@*": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#c14c24f18ea8190c118ee7562b7ff99a36552686" + integrity sha512-plGgXAPfVKFoYfa9NpYDAkseG+g6Jr294RqeqcqDixSbU34MZVJRi/P+7Y8GDpzkEwLaGZZOpKIEmeVZNtKsrg== + dependencies: + "@types/istanbul-lib-coverage" "*" + +"@types/istanbul-reports@^3.0.0": + version "3.0.1" + resolved "https://registry.yarnpkg.com/@types/istanbul-reports/-/istanbul-reports-3.0.1.tgz#9153fe98bba2bd565a63add9436d6f0d7f8468ff" + integrity sha512-c3mAZEuK0lvBp8tmuL74XRKn1+y2dcwOUpH7x4WrF6gk1GIgiluDRgMYQtw2OFcBvAJWlt6ASU3tSqxp0Uu0Aw== + dependencies: + "@types/istanbul-lib-report" "*" + +"@types/node@*": + version "17.0.42" + resolved "https://registry.yarnpkg.com/@types/node/-/node-17.0.42.tgz#d7e8f22700efc94d125103075c074396b5f41f9b" + integrity sha512-Q5BPGyGKcvQgAMbsr7qEGN/kIPN6zZecYYABeTDBizOsau+2NMdSVTar9UQw21A2+JyA2KRNDYaYrPB0Rpk2oQ== + +"@types/normalize-package-data@^2.4.0": + version "2.4.1" + resolved "https://registry.yarnpkg.com/@types/normalize-package-data/-/normalize-package-data-2.4.1.tgz#d3357479a0fdfdd5907fe67e17e0a85c906e1301" + integrity sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw== + +"@types/prettier@^2.0.0": + version "2.6.3" + resolved "https://registry.yarnpkg.com/@types/prettier/-/prettier-2.6.3.tgz#68ada76827b0010d0db071f739314fa429943d0a" + integrity sha512-ymZk3LEC/fsut+/Q5qejp6R9O1rMxz3XaRHDV6kX8MrGAhOSPqVARbDi+EZvInBpw+BnCX3TD240byVkOfQsHg== + +"@types/stack-utils@^2.0.0": + version "2.0.1" + resolved "https://registry.yarnpkg.com/@types/stack-utils/-/stack-utils-2.0.1.tgz#20f18294f797f2209b5f65c8e3b5c8e8261d127c" + integrity sha512-Hl219/BT5fLAaz6NDkSuhzasy49dwQS/DSdu4MdggFB8zcXv7vflBI3xp7FEmkmdDkBUI2bPUNeMttp2knYdxw== + +"@types/yargs-parser@*": + version "21.0.0" + resolved "https://registry.yarnpkg.com/@types/yargs-parser/-/yargs-parser-21.0.0.tgz#0c60e537fa790f5f9472ed2776c2b71ec117351b" + integrity sha512-iO9ZQHkZxHn4mSakYV0vFHAVDyEOIJQrV2uZ06HxEPcx+mt8swXoZHIbaaJ2crJYFfErySgktuTZ3BeLz+XmFA== + +"@types/yargs@^15.0.0": + version "15.0.14" + resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-15.0.14.tgz#26d821ddb89e70492160b66d10a0eb6df8f6fb06" + integrity sha512-yEJzHoxf6SyQGhBhIYGXQDSCkJjB6HohDShto7m8vaKg9Yp0Yn8+71J9eakh2bnPg6BfsH9PRMhiRTZnd4eXGQ== + dependencies: + "@types/yargs-parser" "*" + +abab@^2.0.3, abab@^2.0.5: + version "2.0.6" + resolved "https://registry.yarnpkg.com/abab/-/abab-2.0.6.tgz#41b80f2c871d19686216b82309231cfd3cb3d291" + integrity sha512-j2afSsaIENvHZN2B8GOpF566vZ5WVk5opAiMTvWgaQT8DkbOqsTfvNAvHoRGU2zzP8cPoqys+xHTRDWW8L+/BA== + +acorn-globals@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/acorn-globals/-/acorn-globals-6.0.0.tgz#46cdd39f0f8ff08a876619b55f5ac8a6dc770b45" + integrity sha512-ZQl7LOWaF5ePqqcX4hLuv/bLXYQNfNWw2c0/yX/TsPRKamzHcTGQnlCjHT3TsmkOUVEPS3crCxiPfdzE/Trlhg== + dependencies: + acorn "^7.1.1" + acorn-walk "^7.1.1" + +acorn-walk@^7.1.1: + version "7.2.0" + resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-7.2.0.tgz#0de889a601203909b0fbe07b8938dc21d2e967bc" + integrity sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA== + +acorn@^7.1.1: + version "7.4.1" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-7.4.1.tgz#feaed255973d2e77555b83dbc08851a6c63520fa" + integrity sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A== + +acorn@^8.2.4: + version "8.7.1" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.7.1.tgz#0197122c843d1bf6d0a5e83220a788f278f63c30" + integrity sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A== + +agent-base@6: + version "6.0.2" + resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-6.0.2.tgz#49fff58577cfee3f37176feab4c22e00f86d7f77" + integrity sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ== + dependencies: + debug "4" + +ansi-escapes@^4.2.1: + version "4.3.2" + resolved "https://registry.yarnpkg.com/ansi-escapes/-/ansi-escapes-4.3.2.tgz#6b2291d1db7d98b6521d5f1efa42d0f3a9feb65e" + integrity sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ== + dependencies: + type-fest "^0.21.3" + +ansi-regex@^5.0.0, ansi-regex@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" + integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== + +ansi-styles@^3.2.1: + version "3.2.1" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-3.2.1.tgz#41fbb20243e50b12be0f04b8dedbf07520ce841d" + integrity sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA== + dependencies: + color-convert "^1.9.0" + +ansi-styles@^4.0.0, ansi-styles@^4.1.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-4.3.0.tgz#edd803628ae71c04c85ae7a0906edad34b648937" + integrity sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg== + dependencies: + color-convert "^2.0.1" + +anymatch@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-2.0.0.tgz#bcb24b4f37934d9aa7ac17b4adaf89e7c76ef2eb" + integrity sha512-5teOsQWABXHHBFP9y3skS5P3d/WfWXpv3FUpy+LorMrNYaT9pI4oLMQX7jzQ2KklNpGpWHzdCXTDT2Y3XGlZBw== + dependencies: + micromatch "^3.1.4" + normalize-path "^2.1.1" + +anymatch@^3.0.3, anymatch@~3.1.2: + version "3.1.2" + resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.2.tgz#c0557c096af32f106198f4f4e2a383537e378716" + integrity sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg== + dependencies: + normalize-path "^3.0.0" + picomatch "^2.0.4" + +argparse@^1.0.7: + version "1.0.10" + resolved "https://registry.yarnpkg.com/argparse/-/argparse-1.0.10.tgz#bcd6791ea5ae09725e17e5ad988134cd40b3d911" + integrity sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg== + dependencies: + sprintf-js "~1.0.2" + +arr-diff@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/arr-diff/-/arr-diff-4.0.0.tgz#d6461074febfec71e7e15235761a329a5dc7c520" + integrity sha512-YVIQ82gZPGBebQV/a8dar4AitzCQs0jjXwMPZllpXMaGjXPYVUawSxQrRsjhjupyVxEvbHgUmIhKVlND+j02kA== + +arr-flatten@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/arr-flatten/-/arr-flatten-1.1.0.tgz#36048bbff4e7b47e136644316c99669ea5ae91f1" + integrity sha512-L3hKV5R/p5o81R7O02IGnwpDmkp6E982XhtbuwSe3O4qOtMMMtodicASA1Cny2U+aCXcNpml+m4dPsvsJ3jatg== + +arr-union@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/arr-union/-/arr-union-3.1.0.tgz#e39b09aea9def866a8f206e288af63919bae39c4" + integrity sha512-sKpyeERZ02v1FeCZT8lrfJq5u6goHCtpTAzPwJYe7c8SPFOboNjNg1vz2L4VTn9T4PQxEx13TbXLmYUcS6Ug7Q== + +array-unique@^0.3.2: + version "0.3.2" + resolved "https://registry.yarnpkg.com/array-unique/-/array-unique-0.3.2.tgz#a894b75d4bc4f6cd679ef3244a9fd8f46ae2d428" + integrity sha512-SleRWjh9JUud2wH1hPs9rZBZ33H6T9HOiL0uwGnGx9FpE6wKGyfWugmbkEOIs6qWrZhg0LWeLziLrEwQJhs5mQ== + +assign-symbols@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/assign-symbols/-/assign-symbols-1.0.0.tgz#59667f41fadd4f20ccbc2bb96b8d4f7f78ec0367" + integrity sha512-Q+JC7Whu8HhmTdBph/Tq59IoRtoy6KAm5zzPv00WdujX82lbAL8K7WVjne7vdCsAmbF4AYaDOPyO3k0kl8qIrw== + +asynckit@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/asynckit/-/asynckit-0.4.0.tgz#c79ed97f7f34cb8f2ba1bc9790bcc366474b4b79" + integrity sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q== + +atob@^2.1.2: + version "2.1.2" + resolved "https://registry.yarnpkg.com/atob/-/atob-2.1.2.tgz#6d9517eb9e030d2436666651e86bd9f6f13533c9" + integrity sha512-Wm6ukoaOGJi/73p/cl2GvLjTI5JM1k/O14isD73YML8StrH/7/lRFgmg8nICZgD3bZZvjwCGxtMOD3wWNAu8cg== + +babel-jest@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-26.6.3.tgz#d87d25cb0037577a0c89f82e5755c5d293c01056" + integrity sha512-pl4Q+GAVOHwvjrck6jKjvmGhnO3jHX/xuB9d27f+EJZ/6k+6nMuPjorrYp7s++bKKdANwzElBWnLWaObvTnaZA== + dependencies: + "@jest/transform" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/babel__core" "^7.1.7" + babel-plugin-istanbul "^6.0.0" + babel-preset-jest "^26.6.2" + chalk "^4.0.0" + graceful-fs "^4.2.4" + slash "^3.0.0" + +babel-plugin-dynamic-import-node@^2.3.3: + version "2.3.3" + resolved "https://registry.yarnpkg.com/babel-plugin-dynamic-import-node/-/babel-plugin-dynamic-import-node-2.3.3.tgz#84fda19c976ec5c6defef57f9427b3def66e17a3" + integrity sha512-jZVI+s9Zg3IqA/kdi0i6UDCybUI3aSBLnglhYbSSjKlV7yF1F/5LWv8MakQmvYpnbJDS6fcBL2KzHSxNCMtWSQ== + dependencies: + object.assign "^4.1.0" + +babel-plugin-istanbul@^6.0.0: + version "6.1.1" + resolved "https://registry.yarnpkg.com/babel-plugin-istanbul/-/babel-plugin-istanbul-6.1.1.tgz#fa88ec59232fd9b4e36dbbc540a8ec9a9b47da73" + integrity sha512-Y1IQok9821cC9onCx5otgFfRm7Lm+I+wwxOx738M/WLPZ9Q42m4IG5W0FNX8WLL2gYMZo3JkuXIH2DOpWM+qwA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@istanbuljs/load-nyc-config" "^1.0.0" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-instrument "^5.0.4" + test-exclude "^6.0.0" + +babel-plugin-jest-hoist@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-26.6.2.tgz#8185bd030348d254c6d7dd974355e6a28b21e62d" + integrity sha512-PO9t0697lNTmcEHH69mdtYiOIkkOlj9fySqfO3K1eCcdISevLAE0xY59VLLUj0SoiPiTX/JU2CYFpILydUa5Lw== + dependencies: + "@babel/template" "^7.3.3" + "@babel/types" "^7.3.3" + "@types/babel__core" "^7.0.0" + "@types/babel__traverse" "^7.0.6" + +babel-plugin-polyfill-corejs2@^0.3.0: + version "0.3.1" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.3.1.tgz#440f1b70ccfaabc6b676d196239b138f8a2cfba5" + integrity sha512-v7/T6EQcNfVLfcN2X8Lulb7DjprieyLWJK/zOWH5DUYcAgex9sP3h25Q+DLsX9TloXe3y1O8l2q2Jv9q8UVB9w== + dependencies: + "@babel/compat-data" "^7.13.11" + "@babel/helper-define-polyfill-provider" "^0.3.1" + semver "^6.1.1" + +babel-plugin-polyfill-corejs3@^0.5.0: + version "0.5.2" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.5.2.tgz#aabe4b2fa04a6e038b688c5e55d44e78cd3a5f72" + integrity sha512-G3uJih0XWiID451fpeFaYGVuxHEjzKTHtc9uGFEjR6hHrvNzeS/PX+LLLcetJcytsB5m4j+K3o/EpXJNb/5IEQ== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.3.1" + core-js-compat "^3.21.0" + +babel-plugin-polyfill-regenerator@^0.3.0: + version "0.3.1" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.3.1.tgz#2c0678ea47c75c8cc2fbb1852278d8fb68233990" + integrity sha512-Y2B06tvgHYt1x0yz17jGkGeeMr5FeKUu+ASJ+N6nB5lQ8Dapfg42i0OVrf8PNGJ3zKL4A23snMi1IRwrqqND7A== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.3.1" + +babel-preset-current-node-syntax@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.0.1.tgz#b4399239b89b2a011f9ddbe3e4f401fc40cff73b" + integrity sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ== + dependencies: + "@babel/plugin-syntax-async-generators" "^7.8.4" + "@babel/plugin-syntax-bigint" "^7.8.3" + "@babel/plugin-syntax-class-properties" "^7.8.3" + "@babel/plugin-syntax-import-meta" "^7.8.3" + "@babel/plugin-syntax-json-strings" "^7.8.3" + "@babel/plugin-syntax-logical-assignment-operators" "^7.8.3" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + "@babel/plugin-syntax-numeric-separator" "^7.8.3" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + "@babel/plugin-syntax-top-level-await" "^7.8.3" + +babel-preset-gatsby-package@^1.3.0: + version "1.14.0" + resolved "https://registry.yarnpkg.com/babel-preset-gatsby-package/-/babel-preset-gatsby-package-1.14.0.tgz#cf618427f649bce8baf581db25a6bed4241244a5" + integrity sha512-8GmWYPHedW/p+vGb90x0k/NRhfLevdH2dVwbydsY4jCuQz/3Y4bSkil6wTdUL3aSbah8nvnN6ANFflbLKm143A== + dependencies: + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.14.5" + "@babel/plugin-proposal-optional-chaining" "^7.14.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-transform-runtime" "^7.15.0" + "@babel/plugin-transform-typescript" "^7.15.4" + "@babel/preset-env" "^7.15.4" + "@babel/preset-flow" "^7.14.0" + "@babel/preset-react" "^7.14.0" + "@babel/runtime" "^7.15.4" + babel-plugin-dynamic-import-node "^2.3.3" + core-js "^3.17.2" + +babel-preset-jest@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-26.6.2.tgz#747872b1171df032252426586881d62d31798fee" + integrity sha512-YvdtlVm9t3k777c5NPQIv6cxFFFapys25HiUmuSgHwIZhfifweR5c5Sf5nwE3MAbfu327CYSvps8Yx6ANLyleQ== + dependencies: + babel-plugin-jest-hoist "^26.6.2" + babel-preset-current-node-syntax "^1.0.0" + +balanced-match@^1.0.0: + version "1.0.2" + resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.2.tgz#e83e3a7e3f300b34cb9d87f615fa0cbf357690ee" + integrity sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw== + +base@^0.11.1: + version "0.11.2" + resolved "https://registry.yarnpkg.com/base/-/base-0.11.2.tgz#7bde5ced145b6d551a90db87f83c558b4eb48a8f" + integrity sha512-5T6P4xPgpp0YDFvSWwEZ4NoE3aM4QBQXDzmVbraCkFj8zHM+mba8SyqB5DbZWyR7mYHo6Y7BdQo3MoA4m0TeQg== + dependencies: + cache-base "^1.0.1" + class-utils "^0.3.5" + component-emitter "^1.2.1" + define-property "^1.0.0" + isobject "^3.0.1" + mixin-deep "^1.2.0" + pascalcase "^0.1.1" + +binary-extensions@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/binary-extensions/-/binary-extensions-2.2.0.tgz#75f502eeaf9ffde42fc98829645be4ea76bd9e2d" + integrity sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA== + +bluebird@^3.7.2: + version "3.7.2" + resolved "https://registry.yarnpkg.com/bluebird/-/bluebird-3.7.2.tgz#9f229c15be272454ffa973ace0dbee79a1b0c36f" + integrity sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg== + +brace-expansion@^1.1.7: + version "1.1.11" + resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-1.1.11.tgz#3c7fcbf529d87226f3d2f52b966ff5271eb441dd" + integrity sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA== + dependencies: + balanced-match "^1.0.0" + concat-map "0.0.1" + +braces@^2.3.1: + version "2.3.2" + resolved "https://registry.yarnpkg.com/braces/-/braces-2.3.2.tgz#5979fd3f14cd531565e5fa2df1abfff1dfaee729" + integrity sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w== + dependencies: + arr-flatten "^1.1.0" + array-unique "^0.3.2" + extend-shallow "^2.0.1" + fill-range "^4.0.0" + isobject "^3.0.1" + repeat-element "^1.1.2" + snapdragon "^0.8.1" + snapdragon-node "^2.0.1" + split-string "^3.0.2" + to-regex "^3.0.1" + +braces@^3.0.2, braces@~3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/braces/-/braces-3.0.2.tgz#3454e1a462ee8d599e236df336cd9ea4f8afe107" + integrity sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A== + dependencies: + fill-range "^7.0.1" + +browser-process-hrtime@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/browser-process-hrtime/-/browser-process-hrtime-1.0.0.tgz#3c9b4b7d782c8121e56f10106d84c0d0ffc94626" + integrity sha512-9o5UecI3GhkpM6DrXr69PblIuWxPKk9Y0jHBRhdocZ2y7YECBFCsHm79Pr3OyR2AvjhDkabFJaDJMYRazHgsow== + +browserslist@^4.20.2, browserslist@^4.20.3: + version "4.20.4" + resolved "https://registry.yarnpkg.com/browserslist/-/browserslist-4.20.4.tgz#98096c9042af689ee1e0271333dbc564b8ce4477" + integrity sha512-ok1d+1WpnU24XYN7oC3QWgTyMhY/avPJ/r9T00xxvUOIparA/gc+UPUMaod3i+G6s+nI2nUb9xZ5k794uIwShw== + dependencies: + caniuse-lite "^1.0.30001349" + electron-to-chromium "^1.4.147" + escalade "^3.1.1" + node-releases "^2.0.5" + picocolors "^1.0.0" + +bser@2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/bser/-/bser-2.1.1.tgz#e6787da20ece9d07998533cfd9de6f5c38f4bc05" + integrity sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ== + dependencies: + node-int64 "^0.4.0" + +buffer-from@^1.0.0: + version "1.1.2" + resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5" + integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ== + +cache-base@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/cache-base/-/cache-base-1.0.1.tgz#0a7f46416831c8b662ee36fe4e7c59d76f666ab2" + integrity sha512-AKcdTnFSWATd5/GCPRxr2ChwIJ85CeyrEyjRHlKxQ56d4XJMGym0uAiKn0xbLOGOl3+yRpOTi484dVCEc5AUzQ== + dependencies: + collection-visit "^1.0.0" + component-emitter "^1.2.1" + get-value "^2.0.6" + has-value "^1.0.0" + isobject "^3.0.1" + set-value "^2.0.0" + to-object-path "^0.3.0" + union-value "^1.0.0" + unset-value "^1.0.0" + +call-bind@^1.0.0: + version "1.0.2" + resolved "https://registry.yarnpkg.com/call-bind/-/call-bind-1.0.2.tgz#b1d4e89e688119c3c9a903ad30abb2f6a919be3c" + integrity sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA== + dependencies: + function-bind "^1.1.1" + get-intrinsic "^1.0.2" + +callsites@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" + integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== + +camelcase@^5.0.0, camelcase@^5.3.1: + version "5.3.1" + resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" + integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== + +camelcase@^6.0.0: + version "6.3.0" + resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-6.3.0.tgz#5685b95eb209ac9c0c177467778c9c84df58ba9a" + integrity sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA== + +caniuse-lite@^1.0.30001349: + version "1.0.30001352" + resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001352.tgz#cc6f5da3f983979ad1e2cdbae0505dccaa7c6a12" + integrity sha512-GUgH8w6YergqPQDGWhJGt8GDRnY0L/iJVQcU3eJ46GYf52R8tk0Wxp0PymuFVZboJYXGiCqwozAYZNRjVj6IcA== + +capture-exit@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/capture-exit/-/capture-exit-2.0.0.tgz#fb953bfaebeb781f62898239dabb426d08a509a4" + integrity sha512-PiT/hQmTonHhl/HFGN+Lx3JJUznrVYJ3+AQsnthneZbvW7x+f08Tk7yLJTLEOUvBTbduLeeBkxEaYXUOUrRq6g== + dependencies: + rsvp "^4.8.4" + +chalk@^2.0.0: + version "2.4.2" + resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" + integrity sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ== + dependencies: + ansi-styles "^3.2.1" + escape-string-regexp "^1.0.5" + supports-color "^5.3.0" + +chalk@^4.0.0: + version "4.1.2" + resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.2.tgz#aac4e2b7734a740867aeb16bf02aad556a1e7a01" + integrity sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA== + dependencies: + ansi-styles "^4.1.0" + supports-color "^7.1.0" + +char-regex@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/char-regex/-/char-regex-1.0.2.tgz#d744358226217f981ed58f479b1d6bcc29545dcf" + integrity sha512-kWWXztvZ5SBQV+eRgKFeh8q5sLuZY2+8WUIzlxWVTg+oGwY14qylx1KbKzHd8P6ZYkAg0xyIDU9JMHhyJMZ1jw== + +chokidar@^3.4.0: + version "3.5.3" + resolved "https://registry.yarnpkg.com/chokidar/-/chokidar-3.5.3.tgz#1cf37c8707b932bd1af1ae22c0432e2acd1903bd" + integrity sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw== + dependencies: + anymatch "~3.1.2" + braces "~3.0.2" + glob-parent "~5.1.2" + is-binary-path "~2.1.0" + is-glob "~4.0.1" + normalize-path "~3.0.0" + readdirp "~3.6.0" + optionalDependencies: + fsevents "~2.3.2" + +ci-info@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-2.0.0.tgz#67a9e964be31a51e15e5010d58e6f12834002f46" + integrity sha512-5tK7EtrZ0N+OLFMthtqOj4fI2Jeb88C4CAZPu25LDVUgXJ0A3Js4PMGqrn0JU1W0Mh1/Z8wZzYPxqUrXeBboCQ== + +cjs-module-lexer@^0.6.0: + version "0.6.0" + resolved "https://registry.yarnpkg.com/cjs-module-lexer/-/cjs-module-lexer-0.6.0.tgz#4186fcca0eae175970aee870b9fe2d6cf8d5655f" + integrity sha512-uc2Vix1frTfnuzxxu1Hp4ktSvM3QaI4oXl4ZUqL1wjTu/BGki9TrCWoqLTg/drR1KwAEarXuRFCG2Svr1GxPFw== + +class-utils@^0.3.5: + version "0.3.6" + resolved "https://registry.yarnpkg.com/class-utils/-/class-utils-0.3.6.tgz#f93369ae8b9a7ce02fd41faad0ca83033190c463" + integrity sha512-qOhPa/Fj7s6TY8H8esGu5QNpMMQxz79h+urzrNYN6mn+9BnxlDGf5QZ+XeCDsxSjPqsSR56XOZOJmpeurnLMeg== + dependencies: + arr-union "^3.1.0" + define-property "^0.2.5" + isobject "^3.0.0" + static-extend "^0.1.1" + +cliui@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/cliui/-/cliui-6.0.0.tgz#511d702c0c4e41ca156d7d0e96021f23e13225b1" + integrity sha512-t6wbgtoCXvAzst7QgXxJYqPt0usEfbgQdftEPbLL/cvv6HPE5VgvqCuAIDR0NgU52ds6rFwqrgakNLrHEjCbrQ== + dependencies: + string-width "^4.2.0" + strip-ansi "^6.0.0" + wrap-ansi "^6.2.0" + +co@^4.6.0: + version "4.6.0" + resolved "https://registry.yarnpkg.com/co/-/co-4.6.0.tgz#6ea6bdf3d853ae54ccb8e47bfa0bf3f9031fb184" + integrity sha512-QVb0dM5HvG+uaxitm8wONl7jltx8dqhfU33DcqtOZcLSVIKSDDLDi7+0LbAKiyI8hD9u42m2YxXSkMGWThaecQ== + +collect-v8-coverage@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/collect-v8-coverage/-/collect-v8-coverage-1.0.1.tgz#cc2c8e94fc18bbdffe64d6534570c8a673b27f59" + integrity sha512-iBPtljfCNcTKNAto0KEtDfZ3qzjJvqE3aTGZsbhjSBlorqpXJlaWWtPO35D+ZImoC3KWejX64o+yPGxhWSTzfg== + +collection-visit@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/collection-visit/-/collection-visit-1.0.0.tgz#4bc0373c164bc3291b4d368c829cf1a80a59dca0" + integrity sha512-lNkKvzEeMBBjUGHZ+q6z9pSJla0KWAQPvtzhEV9+iGyQYG+pBpl7xKDhxoNSOZH2hhv0v5k0y2yAM4o4SjoSkw== + dependencies: + map-visit "^1.0.0" + object-visit "^1.0.0" + +color-convert@^1.9.0: + version "1.9.3" + resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" + integrity sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg== + dependencies: + color-name "1.1.3" + +color-convert@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-2.0.1.tgz#72d3a68d598c9bdb3af2ad1e84f21d896abd4de3" + integrity sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ== + dependencies: + color-name "~1.1.4" + +color-name@1.1.3: + version "1.1.3" + resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25" + integrity sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw== + +color-name@~1.1.4: + version "1.1.4" + resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.4.tgz#c2a09a87acbde69543de6f63fa3995c826c536a2" + integrity sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA== + +combined-stream@^1.0.8: + version "1.0.8" + resolved "https://registry.yarnpkg.com/combined-stream/-/combined-stream-1.0.8.tgz#c3d45a8b34fd730631a110a8a2520682b31d5a7f" + integrity sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg== + dependencies: + delayed-stream "~1.0.0" + +commander@^4.0.1: + version "4.1.1" + resolved "https://registry.yarnpkg.com/commander/-/commander-4.1.1.tgz#9fd602bd936294e9e9ef46a3f4d6964044b18068" + integrity sha512-NOKm8xhkzAjzFx8B2v5OAHT+u5pRQc2UCa2Vq9jYL/31o2wi9mxBA7LIFs3sV5VSC49z6pEhfbMULvShKj26WA== + +component-emitter@^1.2.1: + version "1.3.0" + resolved "https://registry.yarnpkg.com/component-emitter/-/component-emitter-1.3.0.tgz#16e4070fba8ae29b679f2215853ee181ab2eabc0" + integrity sha512-Rd3se6QB+sO1TwqZjscQrurpEPIfO0/yYnSin6Q/rD3mOutHvUrCAhJub3r90uNb+SESBuE0QYoB90YdfatsRg== + +concat-map@0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" + integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg== + +convert-source-map@^1.1.0, convert-source-map@^1.4.0, convert-source-map@^1.6.0, convert-source-map@^1.7.0: + version "1.8.0" + resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.8.0.tgz#f3373c32d21b4d780dd8004514684fb791ca4369" + integrity sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA== + dependencies: + safe-buffer "~5.1.1" + +copy-descriptor@^0.1.0: + version "0.1.1" + resolved "https://registry.yarnpkg.com/copy-descriptor/-/copy-descriptor-0.1.1.tgz#676f6eb3c39997c2ee1ac3a924fd6124748f578d" + integrity sha512-XgZ0pFcakEUlbwQEVNg3+QAis1FyTL3Qel9FYy8pSkQqoG3PNoT0bOCQtOXcOkur21r2Eq2kI+IE+gsmAEVlYw== + +core-js-compat@^3.21.0, core-js-compat@^3.22.1: + version "3.22.8" + resolved "https://registry.yarnpkg.com/core-js-compat/-/core-js-compat-3.22.8.tgz#46fa34ce1ddf742acd7f95f575f66bbb21e05d62" + integrity sha512-pQnwg4xtuvc2Bs/5zYQPaEYYSuTxsF7LBWF0SvnVhthZo/Qe+rJpcEekrdNK5DWwDJ0gv0oI9NNX5Mppdy0ctg== + dependencies: + browserslist "^4.20.3" + semver "7.0.0" + +core-js@^3.17.2: + version "3.22.8" + resolved "https://registry.yarnpkg.com/core-js/-/core-js-3.22.8.tgz#23f860b1fe60797cc4f704d76c93fea8a2f60631" + integrity sha512-UoGQ/cfzGYIuiq6Z7vWL1HfkE9U9IZ4Ub+0XSiJTCzvbZzgPA69oDF2f+lgJ6dFFLEdjW5O6svvoKzXX23xFkA== + +cross-env@^7.0.3: + version "7.0.3" + resolved "https://registry.yarnpkg.com/cross-env/-/cross-env-7.0.3.tgz#865264b29677dc015ba8418918965dd232fc54cf" + integrity sha512-+/HKd6EgcQCJGh2PSjZuUitQBQynKor4wrFbRg4DtAgS1aWO+gU52xpH7M9ScGgXSYmAVS9bIJ8EzuaGw0oNAw== + dependencies: + cross-spawn "^7.0.1" + +cross-spawn@^6.0.0: + version "6.0.5" + resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-6.0.5.tgz#4a5ec7c64dfae22c3a14124dbacdee846d80cbc4" + integrity sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ== + dependencies: + nice-try "^1.0.4" + path-key "^2.0.1" + semver "^5.5.0" + shebang-command "^1.2.0" + which "^1.2.9" + +cross-spawn@^7.0.0, cross-spawn@^7.0.1: + version "7.0.3" + resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-7.0.3.tgz#f73a85b9d5d41d045551c177e2882d4ac85728a6" + integrity sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w== + dependencies: + path-key "^3.1.0" + shebang-command "^2.0.0" + which "^2.0.1" + +cssom@^0.4.4: + version "0.4.4" + resolved "https://registry.yarnpkg.com/cssom/-/cssom-0.4.4.tgz#5a66cf93d2d0b661d80bf6a44fb65f5c2e4e0a10" + integrity sha512-p3pvU7r1MyyqbTk+WbNJIgJjG2VmTIaB10rI93LzVPrmDJKkzKYMtxxyAvQXR/NS6otuzveI7+7BBq3SjBS2mw== + +cssom@~0.3.6: + version "0.3.8" + resolved "https://registry.yarnpkg.com/cssom/-/cssom-0.3.8.tgz#9f1276f5b2b463f2114d3f2c75250af8c1a36f4a" + integrity sha512-b0tGHbfegbhPJpxpiBPU2sCkigAqtM9O121le6bbOlgyV+NyGyCmVfJ6QW9eRjz8CpNfWEOYBIMIGRYkLwsIYg== + +cssstyle@^2.3.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/cssstyle/-/cssstyle-2.3.0.tgz#ff665a0ddbdc31864b09647f34163443d90b0852" + integrity sha512-AZL67abkUzIuvcHqk7c09cezpGNcxUxU4Ioi/05xHk4DQeTkWmGYftIE6ctU6AEt+Gn4n1lDStOtj7FKycP71A== + dependencies: + cssom "~0.3.6" + +data-urls@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/data-urls/-/data-urls-2.0.0.tgz#156485a72963a970f5d5821aaf642bef2bf2db9b" + integrity sha512-X5eWTSXO/BJmpdIKCRuKUgSCgAN0OwliVK3yPKbwIWU1Tdw5BRajxlzMidvh+gwko9AfQ9zIj52pzF91Q3YAvQ== + dependencies: + abab "^2.0.3" + whatwg-mimetype "^2.3.0" + whatwg-url "^8.0.0" + +debug@4, debug@^4.1.0, debug@^4.1.1: + version "4.3.4" + resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" + integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== + dependencies: + ms "2.1.2" + +debug@^2.2.0, debug@^2.3.3: + version "2.6.9" + resolved "https://registry.yarnpkg.com/debug/-/debug-2.6.9.tgz#5d128515df134ff327e90a4c93f4e077a536341f" + integrity sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA== + dependencies: + ms "2.0.0" + +decamelize@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290" + integrity sha512-z2S+W9X73hAUUki+N+9Za2lBlun89zigOyGrsax+KUQ6wKW4ZoWpEYBkGhQjwAjjDCkWxhY0VKEhk8wzY7F5cA== + +decimal.js@^10.2.1: + version "10.3.1" + resolved "https://registry.yarnpkg.com/decimal.js/-/decimal.js-10.3.1.tgz#d8c3a444a9c6774ba60ca6ad7261c3a94fd5e783" + integrity sha512-V0pfhfr8suzyPGOx3nmq4aHqabehUZn6Ch9kyFpV79TGDTWFmHqUqXdabR7QHqxzrYolF4+tVmJhUG4OURg5dQ== + +decode-uri-component@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/decode-uri-component/-/decode-uri-component-0.2.0.tgz#eb3913333458775cb84cd1a1fae062106bb87545" + integrity sha512-hjf+xovcEn31w/EUYdTXQh/8smFL/dzYjohQGEIgjyNavaJfBY2p5F527Bo1VPATxv0VYTUC2bOcXvqFwk78Og== + +deep-is@~0.1.3: + version "0.1.4" + resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" + integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== + +deepmerge@^4.2.2: + version "4.2.2" + resolved "https://registry.yarnpkg.com/deepmerge/-/deepmerge-4.2.2.tgz#44d2ea3679b8f4d4ffba33f03d865fc1e7bf4955" + integrity sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg== + +define-properties@^1.1.3: + version "1.1.4" + resolved "https://registry.yarnpkg.com/define-properties/-/define-properties-1.1.4.tgz#0b14d7bd7fbeb2f3572c3a7eda80ea5d57fb05b1" + integrity sha512-uckOqKcfaVvtBdsVkdPv3XjveQJsNQqmhXgRi8uhvWWuPYZCNlzT8qAyblUgNoXdHdjMTzAqeGjAoli8f+bzPA== + dependencies: + has-property-descriptors "^1.0.0" + object-keys "^1.1.1" + +define-property@^0.2.5: + version "0.2.5" + resolved "https://registry.yarnpkg.com/define-property/-/define-property-0.2.5.tgz#c35b1ef918ec3c990f9a5bc57be04aacec5c8116" + integrity sha512-Rr7ADjQZenceVOAKop6ALkkRAmH1A4Gx9hV/7ZujPUN2rkATqFO0JZLZInbAjpZYoJ1gUx8MRMQVkYemcbMSTA== + dependencies: + is-descriptor "^0.1.0" + +define-property@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/define-property/-/define-property-1.0.0.tgz#769ebaaf3f4a63aad3af9e8d304c9bbe79bfb0e6" + integrity sha512-cZTYKFWspt9jZsMscWo8sc/5lbPC9Q0N5nBLgb+Yd915iL3udB1uFgS3B8YCx66UVHq018DAVFoee7x+gxggeA== + dependencies: + is-descriptor "^1.0.0" + +define-property@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/define-property/-/define-property-2.0.2.tgz#d459689e8d654ba77e02a817f8710d702cb16e9d" + integrity sha512-jwK2UV4cnPpbcG7+VRARKTZPUWowwXA8bzH5NP6ud0oeAxyYPuGZUAC7hMugpCdz4BeSZl2Dl9k66CHJ/46ZYQ== + dependencies: + is-descriptor "^1.0.2" + isobject "^3.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" + integrity sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ== + +detect-newline@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/detect-newline/-/detect-newline-3.1.0.tgz#576f5dfc63ae1a192ff192d8ad3af6308991b651" + integrity sha512-TLz+x/vEXm/Y7P7wn1EJFNLxYpUD4TgMosxY6fAVJUnJMbupHBOncxyWUG9OpTaH9EBD7uFI5LfEgmMOc54DsA== + +diff-sequences@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-26.6.2.tgz#48ba99157de1923412eed41db6b6d4aa9ca7c0b1" + integrity sha512-Mv/TDa3nZ9sbc5soK+OoA74BsS3mL37yixCvUAQkiuA4Wz6YtwP/K47n2rv2ovzHZvoiQeA5FTQOschKkEwB0Q== + +domexception@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/domexception/-/domexception-2.0.1.tgz#fb44aefba793e1574b0af6aed2801d057529f304" + integrity sha512-yxJ2mFy/sibVQlu5qHjOkf9J3K6zgmCxgJ94u2EdvDOV09H+32LtRswEcUsmUWN72pVLOEnTSRaIVVzVQgS0dg== + dependencies: + webidl-conversions "^5.0.0" + +electron-to-chromium@^1.4.147: + version "1.4.152" + resolved "https://registry.yarnpkg.com/electron-to-chromium/-/electron-to-chromium-1.4.152.tgz#7dedbe8f3dc1c597088982a203f392e60f7ee90a" + integrity sha512-jk4Ju5SGZAQQJ1iI4Rgru7dDlvkQPLpNPWH9gIZmwCD4YteA5Bbk1xPcPDUf5jUYs3e1e80RXdi8XgKQZaigeg== + +emittery@^0.7.1: + version "0.7.2" + resolved "https://registry.yarnpkg.com/emittery/-/emittery-0.7.2.tgz#25595908e13af0f5674ab419396e2fb394cdfa82" + integrity sha512-A8OG5SR/ij3SsJdWDJdkkSYUjQdCUx6APQXem0SaEePBSRg4eymGYwBkKo1Y6DU+af/Jn2dBQqDBvjnr9Vi8nQ== + +emoji-regex@^8.0.0: + version "8.0.0" + resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" + integrity sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A== + +end-of-stream@^1.1.0: + version "1.4.4" + resolved "https://registry.yarnpkg.com/end-of-stream/-/end-of-stream-1.4.4.tgz#5ae64a5f45057baf3626ec14da0ca5e4b2431eb0" + integrity sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q== + dependencies: + once "^1.4.0" + +error-ex@^1.3.1: + version "1.3.2" + resolved "https://registry.yarnpkg.com/error-ex/-/error-ex-1.3.2.tgz#b4ac40648107fdcdcfae242f428bea8a14d4f1bf" + integrity sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g== + dependencies: + is-arrayish "^0.2.1" + +escalade@^3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/escalade/-/escalade-3.1.1.tgz#d8cfdc7000965c5a0174b4a82eaa5c0552742e40" + integrity sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw== + +escape-string-regexp@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" + integrity sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg== + +escape-string-regexp@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz#a30304e99daa32e23b2fd20f51babd07cffca344" + integrity sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w== + +escodegen@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-2.0.0.tgz#5e32b12833e8aa8fa35e1bf0befa89380484c7dd" + integrity sha512-mmHKys/C8BFUGI+MAWNcSYoORYLMdPzjrknd2Vc+bUsjN5bXcr8EhrNB+UTqfL1y3I9c4fw2ihgtMPQLBRiQxw== + dependencies: + esprima "^4.0.1" + estraverse "^5.2.0" + esutils "^2.0.2" + optionator "^0.8.1" + optionalDependencies: + source-map "~0.6.1" + +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== + +estraverse@^5.2.0: + version "5.3.0" + resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" + integrity sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA== + +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== + +exec-sh@^0.3.2: + version "0.3.6" + resolved "https://registry.yarnpkg.com/exec-sh/-/exec-sh-0.3.6.tgz#ff264f9e325519a60cb5e273692943483cca63bc" + integrity sha512-nQn+hI3yp+oD0huYhKwvYI32+JFeq+XkNcD1GAo3Y/MjxsfVGmrrzrnzjWiNY6f+pUCP440fThsFh5gZrRAU/w== + +execa@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/execa/-/execa-1.0.0.tgz#c6236a5bb4df6d6f15e88e7f017798216749ddd8" + integrity sha512-adbxcyWV46qiHyvSp50TKt05tB4tK3HcmF7/nxfAdhnox83seTDbwnaqKO4sXRy7roHAIFqJP/Rw/AuEbX61LA== + dependencies: + cross-spawn "^6.0.0" + get-stream "^4.0.0" + is-stream "^1.1.0" + npm-run-path "^2.0.0" + p-finally "^1.0.0" + signal-exit "^3.0.0" + strip-eof "^1.0.0" + +execa@^4.0.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/execa/-/execa-4.1.0.tgz#4e5491ad1572f2f17a77d388c6c857135b22847a" + integrity sha512-j5W0//W7f8UxAn8hXVnwG8tLwdiUy4FJLcSupCg6maBYZDpyBvTApK7KyuI4bKj8KOh1r2YH+6ucuYtJv1bTZA== + dependencies: + cross-spawn "^7.0.0" + get-stream "^5.0.0" + human-signals "^1.1.1" + is-stream "^2.0.0" + merge-stream "^2.0.0" + npm-run-path "^4.0.0" + onetime "^5.1.0" + signal-exit "^3.0.2" + strip-final-newline "^2.0.0" + +exit@^0.1.2: + version "0.1.2" + resolved "https://registry.yarnpkg.com/exit/-/exit-0.1.2.tgz#0632638f8d877cc82107d30a0fff1a17cba1cd0c" + integrity sha512-Zk/eNKV2zbjpKzrsQ+n1G6poVbErQxJ0LBOJXaKZ1EViLzH+hrLu9cdXI4zw9dBQJslwBEpbQ2P1oS7nDxs6jQ== + +expand-brackets@^2.1.4: + version "2.1.4" + resolved "https://registry.yarnpkg.com/expand-brackets/-/expand-brackets-2.1.4.tgz#b77735e315ce30f6b6eff0f83b04151a22449622" + integrity sha512-w/ozOKR9Obk3qoWeY/WDi6MFta9AoMR+zud60mdnbniMcBxRuFJyDt2LdX/14A1UABeqk+Uk+LDfUpvoGKppZA== + dependencies: + debug "^2.3.3" + define-property "^0.2.5" + extend-shallow "^2.0.1" + posix-character-classes "^0.1.0" + regex-not "^1.0.0" + snapdragon "^0.8.1" + to-regex "^3.0.1" + +expect@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/expect/-/expect-26.6.2.tgz#c6b996bf26bf3fe18b67b2d0f51fc981ba934417" + integrity sha512-9/hlOBkQl2l/PLHJx6JjoDF6xPKcJEsUlWKb23rKE7KzeDqUZKXKNMW27KIue5JMdBV9HgmoJPcc8HtO85t9IA== + dependencies: + "@jest/types" "^26.6.2" + ansi-styles "^4.0.0" + jest-get-type "^26.3.0" + jest-matcher-utils "^26.6.2" + jest-message-util "^26.6.2" + jest-regex-util "^26.0.0" + +extend-shallow@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/extend-shallow/-/extend-shallow-2.0.1.tgz#51af7d614ad9a9f610ea1bafbb989d6b1c56890f" + integrity sha512-zCnTtlxNoAiDc3gqY2aYAWFx7XWWiasuF2K8Me5WbN8otHKTUKBwjPtNpRs/rbUZm7KxWAaNj7P1a/p52GbVug== + dependencies: + is-extendable "^0.1.0" + +extend-shallow@^3.0.0, extend-shallow@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/extend-shallow/-/extend-shallow-3.0.2.tgz#26a71aaf073b39fb2127172746131c2704028db8" + integrity sha512-BwY5b5Ql4+qZoefgMj2NUmx+tehVTH/Kf4k1ZEtOHNFcm2wSxMRo992l6X3TIgni2eZVTZ85xMOjF31fwZAj6Q== + dependencies: + assign-symbols "^1.0.0" + is-extendable "^1.0.1" + +extglob@^2.0.4: + version "2.0.4" + resolved "https://registry.yarnpkg.com/extglob/-/extglob-2.0.4.tgz#ad00fe4dc612a9232e8718711dc5cb5ab0285543" + integrity sha512-Nmb6QXkELsuBr24CJSkilo6UHHgbekK5UiZgfE6UHD3Eb27YC6oD+bhcT+tJ6cl8dmsgdQxnWlcry8ksBIBLpw== + dependencies: + array-unique "^0.3.2" + define-property "^1.0.0" + expand-brackets "^2.1.4" + extend-shallow "^2.0.1" + fragment-cache "^0.2.1" + regex-not "^1.0.0" + snapdragon "^0.8.1" + to-regex "^3.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" + integrity sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw== + +fast-levenshtein@~2.0.6: + version "2.0.6" + resolved "https://registry.yarnpkg.com/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz#3d8a5c66883a16a30ca8643e851f19baa7797917" + integrity sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw== + +fb-watchman@^2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/fb-watchman/-/fb-watchman-2.0.1.tgz#fc84fb39d2709cf3ff6d743706157bb5708a8a85" + integrity sha512-DkPJKQeY6kKwmuMretBhr7G6Vodr7bFwDYTXIkfG1gjvNpaxBTQV3PbXg6bR1c1UP4jPOX0jHUbbHANL9vRjVg== + dependencies: + bser "2.1.1" + +fill-range@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-4.0.0.tgz#d544811d428f98eb06a63dc402d2403c328c38f7" + integrity sha512-VcpLTWqWDiTerugjj8e3+esbg+skS3M9e54UuR3iCeIDMXCLTsAH8hTSzDQU/X6/6t3eYkOKoZSef2PlU6U1XQ== + dependencies: + extend-shallow "^2.0.1" + is-number "^3.0.0" + repeat-string "^1.6.1" + to-regex-range "^2.1.0" + +fill-range@^7.0.1: + version "7.0.1" + resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-7.0.1.tgz#1919a6a7c75fe38b2c7c77e5198535da9acdda40" + integrity sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ== + dependencies: + to-regex-range "^5.0.1" + +find-up@^4.0.0, find-up@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" + integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== + dependencies: + locate-path "^5.0.0" + path-exists "^4.0.0" + +for-in@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/for-in/-/for-in-1.0.2.tgz#81068d295a8142ec0ac726c6e2200c30fb6d5e80" + integrity sha512-7EwmXrOjyL+ChxMhmG5lnW9MPt1aIeZEwKhQzoBUdTV0N3zuwWDZYVJatDvZ2OyzPUvdIAZDsCetk3coyMfcnQ== + +form-data@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/form-data/-/form-data-3.0.1.tgz#ebd53791b78356a99af9a300d4282c4d5eb9755f" + integrity sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg== + dependencies: + asynckit "^0.4.0" + combined-stream "^1.0.8" + mime-types "^2.1.12" + +fragment-cache@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/fragment-cache/-/fragment-cache-0.2.1.tgz#4290fad27f13e89be7f33799c6bc5a0abfff0d19" + integrity sha512-GMBAbW9antB8iZRHLoGw0b3HANt57diZYFO/HL1JGIC1MjKrdmhxvrJbupnVvpys0zsz7yBApXdQyfepKly2kA== + dependencies: + map-cache "^0.2.2" + +fs-readdir-recursive@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/fs-readdir-recursive/-/fs-readdir-recursive-1.1.0.tgz#e32fc030a2ccee44a6b5371308da54be0b397d27" + integrity sha512-GNanXlVr2pf02+sPN40XN8HG+ePaNcvM0q5mZBd668Obwb0yD5GiUbZOFgwn8kGMY6I3mdyDJzieUy3PTYyTRA== + +fs.realpath@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" + integrity sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw== + +fsevents@^2.1.2, fsevents@~2.3.2: + version "2.3.2" + resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.2.tgz#8a526f78b8fdf4623b709e0b975c52c24c02fd1a" + integrity sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA== + +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== + +gensync@^1.0.0-beta.2: + version "1.0.0-beta.2" + resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" + integrity sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg== + +get-caller-file@^2.0.1: + version "2.0.5" + resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" + integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== + +get-intrinsic@^1.0.2, get-intrinsic@^1.1.1: + version "1.1.2" + resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.1.2.tgz#336975123e05ad0b7ba41f152ee4aadbea6cf598" + integrity sha512-Jfm3OyCxHh9DJyc28qGk+JmfkpO41A4XkneDSujN9MDXrm4oDKdHvndhZ2dN94+ERNfkYJWDclW6k2L/ZGHjXA== + dependencies: + function-bind "^1.1.1" + has "^1.0.3" + has-symbols "^1.0.3" + +get-package-type@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/get-package-type/-/get-package-type-0.1.0.tgz#8de2d803cff44df3bc6c456e6668b36c3926e11a" + integrity sha512-pjzuKtY64GYfWizNAJ0fr9VqttZkNiK2iS430LtIHzjBEr6bX8Am2zm4sW4Ro5wjWW5cAlRL1qAMTcXbjNAO2Q== + +get-stream@^4.0.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-4.1.0.tgz#c1b255575f3dc21d59bfc79cd3d2b46b1c3a54b5" + integrity sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w== + dependencies: + pump "^3.0.0" + +get-stream@^5.0.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-5.2.0.tgz#4966a1795ee5ace65e706c4b7beb71257d6e22d3" + integrity sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA== + dependencies: + pump "^3.0.0" + +get-value@^2.0.3, get-value@^2.0.6: + version "2.0.6" + resolved "https://registry.yarnpkg.com/get-value/-/get-value-2.0.6.tgz#dc15ca1c672387ca76bd37ac0a395ba2042a2c28" + integrity sha512-Ln0UQDlxH1BapMu3GPtf7CuYNwRZf2gwCuPqbyG6pB8WfmFpzqcy4xtAaAMUhnNqjMKTiCPZG2oMT3YSx8U2NA== + +glob-parent@~5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-5.1.2.tgz#869832c58034fe68a4093c17dc15e8340d8401c4" + integrity sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow== + dependencies: + is-glob "^4.0.1" + +glob@^7.0.0, glob@^7.1.1, glob@^7.1.2, glob@^7.1.3, glob@^7.1.4: + version "7.2.3" + resolved "https://registry.yarnpkg.com/glob/-/glob-7.2.3.tgz#b8df0fb802bbfa8e89bd1d938b4e16578ed44f2b" + integrity sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.1.1" + once "^1.3.0" + path-is-absolute "^1.0.0" + +globals@^11.1.0: + version "11.12.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-11.12.0.tgz#ab8795338868a0babd8525758018c2a7eb95c42e" + integrity sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA== + +graceful-fs@^4.2.4: + version "4.2.10" + resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.10.tgz#147d3a006da4ca3ce14728c7aefc287c367d7a6c" + integrity sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA== + +growly@^1.3.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/growly/-/growly-1.3.0.tgz#f10748cbe76af964b7c96c93c6bcc28af120c081" + integrity sha512-+xGQY0YyAWCnqy7Cd++hc2JqMYzlm0dG30Jd0beaA64sROr8C4nt8Yc9V5Ro3avlSUDTN0ulqP/VBKi1/lLygw== + +has-flag@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-3.0.0.tgz#b5d454dc2199ae225699f3467e5a07f3b955bafd" + integrity sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw== + +has-flag@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b" + integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ== + +has-property-descriptors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz#610708600606d36961ed04c196193b6a607fa861" + integrity sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ== + dependencies: + get-intrinsic "^1.1.1" + +has-symbols@^1.0.1, has-symbols@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.3.tgz#bb7b2c4349251dce87b125f7bdf874aa7c8b39f8" + integrity sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A== + +has-value@^0.3.1: + version "0.3.1" + resolved "https://registry.yarnpkg.com/has-value/-/has-value-0.3.1.tgz#7b1f58bada62ca827ec0a2078025654845995e1f" + integrity sha512-gpG936j8/MzaeID5Yif+577c17TxaDmhuyVgSwtnL/q8UUTySg8Mecb+8Cf1otgLoD7DDH75axp86ER7LFsf3Q== + dependencies: + get-value "^2.0.3" + has-values "^0.1.4" + isobject "^2.0.0" + +has-value@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/has-value/-/has-value-1.0.0.tgz#18b281da585b1c5c51def24c930ed29a0be6b177" + integrity sha512-IBXk4GTsLYdQ7Rvt+GRBrFSVEkmuOUy4re0Xjd9kJSUQpnTrWR4/y9RpfexN9vkAPMFuQoeWKwqzPozRTlasGw== + dependencies: + get-value "^2.0.6" + has-values "^1.0.0" + isobject "^3.0.0" + +has-values@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/has-values/-/has-values-0.1.4.tgz#6d61de95d91dfca9b9a02089ad384bff8f62b771" + integrity sha512-J8S0cEdWuQbqD9//tlZxiMuMNmxB8PlEwvYwuxsTmR1G5RXUePEX/SJn7aD0GMLieuZYSwNH0cQuJGwnYunXRQ== + +has-values@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/has-values/-/has-values-1.0.0.tgz#95b0b63fec2146619a6fe57fe75628d5a39efe4f" + integrity sha512-ODYZC64uqzmtfGMEAX/FvZiRyWLpAC3vYnNunURUnkGVTS+mI0smVsWaPydRBsE3g+ok7h960jChO8mFcWlHaQ== + dependencies: + is-number "^3.0.0" + kind-of "^4.0.0" + +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== + dependencies: + function-bind "^1.1.1" + +hosted-git-info@^2.1.4: + version "2.8.9" + resolved "https://registry.yarnpkg.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz#dffc0bf9a21c02209090f2aa69429e1414daf3f9" + integrity sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw== + +html-encoding-sniffer@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/html-encoding-sniffer/-/html-encoding-sniffer-2.0.1.tgz#42a6dc4fd33f00281176e8b23759ca4e4fa185f3" + integrity sha512-D5JbOMBIR/TVZkubHT+OyT2705QvogUW4IBn6nHd756OwieSF9aDYFj4dv6HHEVGYbHaLETa3WggZYWWMyy3ZQ== + dependencies: + whatwg-encoding "^1.0.5" + +html-escaper@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/html-escaper/-/html-escaper-2.0.2.tgz#dfd60027da36a36dfcbe236262c00a5822681453" + integrity sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg== + +http-proxy-agent@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/http-proxy-agent/-/http-proxy-agent-4.0.1.tgz#8a8c8ef7f5932ccf953c296ca8291b95aa74aa3a" + integrity sha512-k0zdNgqWTGA6aeIRVpvfVob4fL52dTfaehylg0Y4UvSySvOq/Y+BOyPrgpUrA7HylqvU8vIZGsRuXmspskV0Tg== + dependencies: + "@tootallnate/once" "1" + agent-base "6" + debug "4" + +https-proxy-agent@^5.0.0: + version "5.0.1" + resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz#c59ef224a04fe8b754f3db0063a25ea30d0005d6" + integrity sha512-dFcAjpTQFgoLMzC2VwU+C/CbS7uRL0lWmxDITmqm7C+7F0Odmj6s9l6alZc6AELXhrnggM2CeWSXHGOdX2YtwA== + dependencies: + agent-base "6" + debug "4" + +human-signals@^1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-1.1.1.tgz#c5b1cd14f50aeae09ab6c59fe63ba3395fe4dfa3" + integrity sha512-SEQu7vl8KjNL2eoGBLF3+wAjpsNfA9XMlXAYj/3EdaNfAlxKthD1xjEQfGOUhllCGGJVNY34bRr6lPINhNjyZw== + +iconv-lite@0.4.24: + version "0.4.24" + resolved "https://registry.yarnpkg.com/iconv-lite/-/iconv-lite-0.4.24.tgz#2022b4b25fbddc21d2f524974a474aafe733908b" + integrity sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA== + dependencies: + safer-buffer ">= 2.1.2 < 3" + +import-local@^3.0.2: + version "3.1.0" + resolved "https://registry.yarnpkg.com/import-local/-/import-local-3.1.0.tgz#b4479df8a5fd44f6cdce24070675676063c95cb4" + integrity sha512-ASB07uLtnDs1o6EHjKpX34BKYDSqnFerfTOJL2HvMqF70LnxpjkzDB8J44oT9pu4AMPkQwf8jl6szgvNd2tRIg== + dependencies: + pkg-dir "^4.2.0" + resolve-cwd "^3.0.0" + +imurmurhash@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" + integrity sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA== + +inflight@^1.0.4: + version "1.0.6" + resolved "https://registry.yarnpkg.com/inflight/-/inflight-1.0.6.tgz#49bd6331d7d02d0c09bc910a1075ba8165b56df9" + integrity sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA== + dependencies: + once "^1.3.0" + wrappy "1" + +inherits@2: + version "2.0.4" + resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" + integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== + +is-accessor-descriptor@^0.1.6: + version "0.1.6" + resolved "https://registry.yarnpkg.com/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz#a9e12cb3ae8d876727eeef3843f8a0897b5c98d6" + integrity sha512-e1BM1qnDbMRG3ll2U9dSK0UMHuWOs3pY3AtcFsmvwPtKL3MML/Q86i+GilLfvqEs4GW+ExB91tQ3Ig9noDIZ+A== + dependencies: + kind-of "^3.0.2" + +is-accessor-descriptor@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz#169c2f6d3df1f992618072365c9b0ea1f6878656" + integrity sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ== + dependencies: + kind-of "^6.0.0" + +is-arrayish@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.2.1.tgz#77c99840527aa8ecb1a8ba697b80645a7a926a9d" + integrity sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg== + +is-binary-path@~2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/is-binary-path/-/is-binary-path-2.1.0.tgz#ea1f7f3b80f064236e83470f86c09c254fb45b09" + integrity sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw== + dependencies: + binary-extensions "^2.0.0" + +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-ci@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/is-ci/-/is-ci-2.0.0.tgz#6bc6334181810e04b5c22b3d589fdca55026404c" + integrity sha512-YfJT7rkpQB0updsdHLGWrvhBJfcfzNNawYDNIyQXJz0IViGf75O8EBPKSdvw2rF+LGCsX4FZ8tcr3b19LcZq4w== + dependencies: + ci-info "^2.0.0" + +is-core-module@^2.8.1: + version "2.9.0" + resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.9.0.tgz#e1c34429cd51c6dd9e09e0799e396e27b19a9c69" + integrity sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A== + dependencies: + has "^1.0.3" + +is-data-descriptor@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz#0b5ee648388e2c860282e793f1856fec3f301b56" + integrity sha512-+w9D5ulSoBNlmw9OHn3U2v51SyoCd0he+bB3xMl62oijhrspxowjU+AIcDY0N3iEJbUEkB15IlMASQsxYigvXg== + dependencies: + kind-of "^3.0.2" + +is-data-descriptor@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz#d84876321d0e7add03990406abbbbd36ba9268c7" + integrity sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ== + dependencies: + kind-of "^6.0.0" + +is-descriptor@^0.1.0: + version "0.1.6" + resolved "https://registry.yarnpkg.com/is-descriptor/-/is-descriptor-0.1.6.tgz#366d8240dde487ca51823b1ab9f07a10a78251ca" + integrity sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg== + dependencies: + is-accessor-descriptor "^0.1.6" + is-data-descriptor "^0.1.4" + kind-of "^5.0.0" + +is-descriptor@^1.0.0, is-descriptor@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-descriptor/-/is-descriptor-1.0.2.tgz#3b159746a66604b04f8c81524ba365c5f14d86ec" + integrity sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg== + dependencies: + is-accessor-descriptor "^1.0.0" + is-data-descriptor "^1.0.0" + kind-of "^6.0.2" + +is-docker@^2.0.0: + version "2.2.1" + resolved "https://registry.yarnpkg.com/is-docker/-/is-docker-2.2.1.tgz#33eeabe23cfe86f14bde4408a02c0cfb853acdaa" + integrity sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ== + +is-extendable@^0.1.0, is-extendable@^0.1.1: + version "0.1.1" + resolved "https://registry.yarnpkg.com/is-extendable/-/is-extendable-0.1.1.tgz#62b110e289a471418e3ec36a617d472e301dfc89" + integrity sha512-5BMULNob1vgFX6EjQw5izWDxrecWK9AM72rugNr0TFldMOi0fj6Jk+zeKIt0xGj4cEfQIJth4w3OKWOJ4f+AFw== + +is-extendable@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/is-extendable/-/is-extendable-1.0.1.tgz#a7470f9e426733d81bd81e1155264e3a3507cab4" + integrity sha512-arnXMxT1hhoKo9k1LZdmlNyJdDDfy2v0fXjFlmok4+i8ul/6WlbVge9bhM74OpNPQPMGUToDtz+KXa1PneJxOA== + dependencies: + is-plain-object "^2.0.4" + +is-extglob@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2" + integrity sha512-SbKbANkN603Vi4jEZv49LeVJMn4yGwsbzZworEoyEiutsN3nJYdbO36zfhGJ6QEDpOZIFkDtnq5JRxmvl3jsoQ== + +is-fullwidth-code-point@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" + integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== + +is-generator-fn@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/is-generator-fn/-/is-generator-fn-2.1.0.tgz#7d140adc389aaf3011a8f2a2a4cfa6faadffb118" + integrity sha512-cTIB4yPYL/Grw0EaSzASzg6bBy9gqCofvWN8okThAYIxKJZC+udlRAmGbM0XLeniEJSs8uEgHPGuHSe1XsOLSQ== + +is-glob@^4.0.1, is-glob@~4.0.1: + version "4.0.3" + resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" + integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg== + dependencies: + is-extglob "^2.1.1" + +is-number@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/is-number/-/is-number-3.0.0.tgz#24fd6201a4782cf50561c810276afc7d12d71195" + integrity sha512-4cboCqIpliH+mAvFNegjZQ4kgKc3ZUhQVr3HvWbSh5q3WH2v82ct+T2Y1hdU5Gdtorx/cLifQjqCbL7bpznLTg== + dependencies: + kind-of "^3.0.2" + +is-number@^7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" + integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== + +is-plain-object@^2.0.3, is-plain-object@^2.0.4: + version "2.0.4" + resolved "https://registry.yarnpkg.com/is-plain-object/-/is-plain-object-2.0.4.tgz#2c163b3fafb1b606d9d17928f05c2a1c38e07677" + integrity sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og== + dependencies: + isobject "^3.0.1" + +is-potential-custom-element-name@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/is-potential-custom-element-name/-/is-potential-custom-element-name-1.0.1.tgz#171ed6f19e3ac554394edf78caa05784a45bebb5" + integrity sha512-bCYeRA2rVibKZd+s2625gGnGF/t7DSqDs4dP7CrLA1m7jKWz6pps0LpYLJN8Q64HtmPKJ1hrN3nzPNKFEKOUiQ== + +is-stream@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-1.1.0.tgz#12d4a3dd4e68e0b79ceb8dbc84173ae80d91ca44" + integrity sha512-uQPm8kcs47jx38atAcWTVxyltQYoPT68y9aWYdV6yWXSyW8mzSat0TL6CiWdZeCdF3KrAvpVtnHbTv4RN+rqdQ== + +is-stream@^2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077" + integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg== + +is-typedarray@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-typedarray/-/is-typedarray-1.0.0.tgz#e479c80858df0c1b11ddda6940f96011fcda4a9a" + integrity sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA== + +is-windows@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-windows/-/is-windows-1.0.2.tgz#d1850eb9791ecd18e6182ce12a30f396634bb19d" + integrity sha512-eXK1UInq2bPmjyX6e3VHIzMLobc4J94i4AWn+Hpq3OU5KkrRC96OAcR3PRJ/pGu6m8TRnBHP9dkXQVsT/COVIA== + +is-wsl@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/is-wsl/-/is-wsl-2.2.0.tgz#74a4c76e77ca9fd3f932f290c17ea326cd157271" + integrity sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww== + dependencies: + is-docker "^2.0.0" + +isarray@1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" + integrity sha512-VLghIWNM6ELQzo7zwmcg0NmTVyWKYjvIeM83yjp0wRDTmUnrM678fQbcKBo6n2CJEF0szoG//ytg+TKla89ALQ== + +isexe@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" + integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== + +isobject@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/isobject/-/isobject-2.1.0.tgz#f065561096a3f1da2ef46272f815c840d87e0c89" + integrity sha512-+OUdGJlgjOBZDfxnDjYYG6zp487z0JGNQq3cYQYg5f5hKR+syHMsaztzGeml/4kGG55CSpKSpWTY+jYGgsHLgA== + dependencies: + isarray "1.0.0" + +isobject@^3.0.0, isobject@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/isobject/-/isobject-3.0.1.tgz#4e431e92b11a9731636aa1f9c8d1ccbcfdab78df" + integrity sha512-WhB9zCku7EGTj/HQQRz5aUQEUeoQZH2bWcltRErOpymJ4boYE6wL9Tbr23krRPSZ+C5zqNSrSw+Cc7sZZ4b7vg== + +istanbul-lib-coverage@^3.0.0, istanbul-lib-coverage@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.0.tgz#189e7909d0a39fa5a3dfad5b03f71947770191d3" + integrity sha512-eOeJ5BHCmHYvQK7xt9GkdHuzuCGS1Y6g9Gvnx3Ym33fz/HpLRYxiS0wHNr+m/MBC8B647Xt608vCDEvhl9c6Mw== + +istanbul-lib-instrument@^4.0.3: + version "4.0.3" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-4.0.3.tgz#873c6fff897450118222774696a3f28902d77c1d" + integrity sha512-BXgQl9kf4WTCPCCpmFGoJkz/+uhvm7h7PFKUYxh7qarQd3ER33vHG//qaE8eN25l07YqZPpHXU9I09l/RD5aGQ== + dependencies: + "@babel/core" "^7.7.5" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-coverage "^3.0.0" + semver "^6.3.0" + +istanbul-lib-instrument@^5.0.4: + version "5.2.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-5.2.0.tgz#31d18bdd127f825dd02ea7bfdfd906f8ab840e9f" + integrity sha512-6Lthe1hqXHBNsqvgDzGO6l03XNeu3CrG4RqQ1KM9+l5+jNGpEJfIELx1NS3SEHmJQA8np/u+E4EPRKRiu6m19A== + dependencies: + "@babel/core" "^7.12.3" + "@babel/parser" "^7.14.7" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-coverage "^3.2.0" + semver "^6.3.0" + +istanbul-lib-report@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#7518fe52ea44de372f460a76b5ecda9ffb73d8a6" + integrity sha512-wcdi+uAKzfiGT2abPpKZ0hSU1rGQjUQnLvtY5MpQ7QCTahD3VODhcu4wcfY1YtkGaDD5yuydOLINXsfbus9ROw== + dependencies: + istanbul-lib-coverage "^3.0.0" + make-dir "^3.0.0" + supports-color "^7.1.0" + +istanbul-lib-source-maps@^4.0.0: + version "4.0.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-source-maps/-/istanbul-lib-source-maps-4.0.1.tgz#895f3a709fcfba34c6de5a42939022f3e4358551" + integrity sha512-n3s8EwkdFIJCG3BPKBYvskgXGoy88ARzvegkitk60NxRdwltLOTaH7CUiMRXvwYorl0Q712iEjcWB+fK/MrWVw== + dependencies: + debug "^4.1.1" + istanbul-lib-coverage "^3.0.0" + source-map "^0.6.1" + +istanbul-reports@^3.0.2: + version "3.1.4" + resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-3.1.4.tgz#1b6f068ecbc6c331040aab5741991273e609e40c" + integrity sha512-r1/DshN4KSE7xWEknZLLLLDn5CJybV3nw01VTkp6D5jzLuELlcbudfj/eSQFvrKsJuTVCGnePO7ho82Nw9zzfw== + dependencies: + html-escaper "^2.0.0" + istanbul-lib-report "^3.0.0" + +jest-changed-files@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-changed-files/-/jest-changed-files-26.6.2.tgz#f6198479e1cc66f22f9ae1e22acaa0b429c042d0" + integrity sha512-fDS7szLcY9sCtIip8Fjry9oGf3I2ht/QT21bAHm5Dmf0mD4X3ReNUf17y+bO6fR8WgbIZTlbyG1ak/53cbRzKQ== + dependencies: + "@jest/types" "^26.6.2" + execa "^4.0.0" + throat "^5.0.0" + +jest-cli@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-cli/-/jest-cli-26.6.3.tgz#43117cfef24bc4cd691a174a8796a532e135e92a" + integrity sha512-GF9noBSa9t08pSyl3CY4frMrqp+aQXFGFkf5hEPbh/pIUFYWMK6ZLTfbmadxJVcJrdRoChlWQsA2VkJcDFK8hg== + dependencies: + "@jest/core" "^26.6.3" + "@jest/test-result" "^26.6.2" + "@jest/types" "^26.6.2" + chalk "^4.0.0" + exit "^0.1.2" + graceful-fs "^4.2.4" + import-local "^3.0.2" + is-ci "^2.0.0" + jest-config "^26.6.3" + jest-util "^26.6.2" + jest-validate "^26.6.2" + prompts "^2.0.1" + yargs "^15.4.1" + +jest-config@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-26.6.3.tgz#64f41444eef9eb03dc51d5c53b75c8c71f645349" + integrity sha512-t5qdIj/bCj2j7NFVHb2nFB4aUdfucDn3JRKgrZnplb8nieAirAzRSHP8uDEd+qV6ygzg9Pz4YG7UTJf94LPSyg== + dependencies: + "@babel/core" "^7.1.0" + "@jest/test-sequencer" "^26.6.3" + "@jest/types" "^26.6.2" + babel-jest "^26.6.3" + chalk "^4.0.0" + deepmerge "^4.2.2" + glob "^7.1.1" + graceful-fs "^4.2.4" + jest-environment-jsdom "^26.6.2" + jest-environment-node "^26.6.2" + jest-get-type "^26.3.0" + jest-jasmine2 "^26.6.3" + jest-regex-util "^26.0.0" + jest-resolve "^26.6.2" + jest-util "^26.6.2" + jest-validate "^26.6.2" + micromatch "^4.0.2" + pretty-format "^26.6.2" + +jest-diff@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-26.6.2.tgz#1aa7468b52c3a68d7d5c5fdcdfcd5e49bd164394" + integrity sha512-6m+9Z3Gv9wN0WFVasqjCL/06+EFCMTqDEUl/b87HYK2rAPTyfz4ZIuSlPhY51PIQRWx5TaxeF1qmXKe9gfN3sA== + dependencies: + chalk "^4.0.0" + diff-sequences "^26.6.2" + jest-get-type "^26.3.0" + pretty-format "^26.6.2" + +jest-docblock@^26.0.0: + version "26.0.0" + resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-26.0.0.tgz#3e2fa20899fc928cb13bd0ff68bd3711a36889b5" + integrity sha512-RDZ4Iz3QbtRWycd8bUEPxQsTlYazfYn/h5R65Fc6gOfwozFhoImx+affzky/FFBuqISPTqjXomoIGJVKBWoo0w== + dependencies: + detect-newline "^3.0.0" + +jest-each@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-each/-/jest-each-26.6.2.tgz#02526438a77a67401c8a6382dfe5999952c167cb" + integrity sha512-Mer/f0KaATbjl8MCJ+0GEpNdqmnVmDYqCTJYTvoo7rqmRiDllmp2AYN+06F93nXcY3ur9ShIjS+CO/uD+BbH4A== + dependencies: + "@jest/types" "^26.6.2" + chalk "^4.0.0" + jest-get-type "^26.3.0" + jest-util "^26.6.2" + pretty-format "^26.6.2" + +jest-environment-jsdom@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-environment-jsdom/-/jest-environment-jsdom-26.6.2.tgz#78d09fe9cf019a357009b9b7e1f101d23bd1da3e" + integrity sha512-jgPqCruTlt3Kwqg5/WVFyHIOJHsiAvhcp2qiR2QQstuG9yWox5+iHpU3ZrcBxW14T4fe5Z68jAfLRh7joCSP2Q== + dependencies: + "@jest/environment" "^26.6.2" + "@jest/fake-timers" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + jest-mock "^26.6.2" + jest-util "^26.6.2" + jsdom "^16.4.0" + +jest-environment-node@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-26.6.2.tgz#824e4c7fb4944646356f11ac75b229b0035f2b0c" + integrity sha512-zhtMio3Exty18dy8ee8eJ9kjnRyZC1N4C1Nt/VShN1apyXc8rWGtJ9lI7vqiWcyyXS4BVSEn9lxAM2D+07/Tag== + dependencies: + "@jest/environment" "^26.6.2" + "@jest/fake-timers" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + jest-mock "^26.6.2" + jest-util "^26.6.2" + +jest-get-type@^26.3.0: + version "26.3.0" + resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-26.3.0.tgz#e97dc3c3f53c2b406ca7afaed4493b1d099199e0" + integrity sha512-TpfaviN1R2pQWkIihlfEanwOXK0zcxrKEE4MlU6Tn7keoXdN6/3gK/xl0yEh8DOunn5pOVGKf8hB4R9gVh04ig== + +jest-haste-map@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-26.6.2.tgz#dd7e60fe7dc0e9f911a23d79c5ff7fb5c2cafeaa" + integrity sha512-easWIJXIw71B2RdR8kgqpjQrbMRWQBgiBwXYEhtGUTaX+doCjBheluShdDMeR8IMfJiTqH4+zfhtg29apJf/8w== + dependencies: + "@jest/types" "^26.6.2" + "@types/graceful-fs" "^4.1.2" + "@types/node" "*" + anymatch "^3.0.3" + fb-watchman "^2.0.0" + graceful-fs "^4.2.4" + jest-regex-util "^26.0.0" + jest-serializer "^26.6.2" + jest-util "^26.6.2" + jest-worker "^26.6.2" + micromatch "^4.0.2" + sane "^4.0.3" + walker "^1.0.7" + optionalDependencies: + fsevents "^2.1.2" + +jest-jasmine2@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-jasmine2/-/jest-jasmine2-26.6.3.tgz#adc3cf915deacb5212c93b9f3547cd12958f2edd" + integrity sha512-kPKUrQtc8aYwBV7CqBg5pu+tmYXlvFlSFYn18ev4gPFtrRzB15N2gW/Roew3187q2w2eHuu0MU9TJz6w0/nPEg== + dependencies: + "@babel/traverse" "^7.1.0" + "@jest/environment" "^26.6.2" + "@jest/source-map" "^26.6.2" + "@jest/test-result" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + chalk "^4.0.0" + co "^4.6.0" + expect "^26.6.2" + is-generator-fn "^2.0.0" + jest-each "^26.6.2" + jest-matcher-utils "^26.6.2" + jest-message-util "^26.6.2" + jest-runtime "^26.6.3" + jest-snapshot "^26.6.2" + jest-util "^26.6.2" + pretty-format "^26.6.2" + throat "^5.0.0" + +jest-leak-detector@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-26.6.2.tgz#7717cf118b92238f2eba65054c8a0c9c653a91af" + integrity sha512-i4xlXpsVSMeKvg2cEKdfhh0H39qlJlP5Ex1yQxwF9ubahboQYMgTtz5oML35AVA3B4Eu+YsmwaiKVev9KCvLxg== + dependencies: + jest-get-type "^26.3.0" + pretty-format "^26.6.2" + +jest-matcher-utils@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-26.6.2.tgz#8e6fd6e863c8b2d31ac6472eeb237bc595e53e7a" + integrity sha512-llnc8vQgYcNqDrqRDXWwMr9i7rS5XFiCwvh6DTP7Jqa2mqpcCBBlpCbn+trkG0KNhPu/h8rzyBkriOtBstvWhw== + dependencies: + chalk "^4.0.0" + jest-diff "^26.6.2" + jest-get-type "^26.3.0" + pretty-format "^26.6.2" + +jest-message-util@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-26.6.2.tgz#58173744ad6fc0506b5d21150b9be56ef001ca07" + integrity sha512-rGiLePzQ3AzwUshu2+Rn+UMFk0pHN58sOG+IaJbk5Jxuqo3NYO1U2/MIR4S1sKgsoYSXSzdtSa0TgrmtUwEbmA== + dependencies: + "@babel/code-frame" "^7.0.0" + "@jest/types" "^26.6.2" + "@types/stack-utils" "^2.0.0" + chalk "^4.0.0" + graceful-fs "^4.2.4" + micromatch "^4.0.2" + pretty-format "^26.6.2" + slash "^3.0.0" + stack-utils "^2.0.2" + +jest-mock@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-26.6.2.tgz#d6cb712b041ed47fe0d9b6fc3474bc6543feb302" + integrity sha512-YyFjePHHp1LzpzYcmgqkJ0nm0gg/lJx2aZFzFy1S6eUqNjXsOqTK10zNRff2dNfssgokjkG65OlWNcIlgd3zew== + dependencies: + "@jest/types" "^26.6.2" + "@types/node" "*" + +jest-pnp-resolver@^1.2.2: + version "1.2.2" + resolved "https://registry.yarnpkg.com/jest-pnp-resolver/-/jest-pnp-resolver-1.2.2.tgz#b704ac0ae028a89108a4d040b3f919dfddc8e33c" + integrity sha512-olV41bKSMm8BdnuMsewT4jqlZ8+3TCARAXjZGT9jcoSnrfUnRCqnMoF9XEeoWjbzObpqF9dRhHQj0Xb9QdF6/w== + +jest-regex-util@^26.0.0: + version "26.0.0" + resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-26.0.0.tgz#d25e7184b36e39fd466c3bc41be0971e821fee28" + integrity sha512-Gv3ZIs/nA48/Zvjrl34bf+oD76JHiGDUxNOVgUjh3j890sblXryjY4rss71fPtD/njchl6PSE2hIhvyWa1eT0A== + +jest-resolve-dependencies@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-resolve-dependencies/-/jest-resolve-dependencies-26.6.3.tgz#6680859ee5d22ee5dcd961fe4871f59f4c784fb6" + integrity sha512-pVwUjJkxbhe4RY8QEWzN3vns2kqyuldKpxlxJlzEYfKSvY6/bMvxoFrYYzUO1Gx28yKWN37qyV7rIoIp2h8fTg== + dependencies: + "@jest/types" "^26.6.2" + jest-regex-util "^26.0.0" + jest-snapshot "^26.6.2" + +jest-resolve@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-26.6.2.tgz#a3ab1517217f469b504f1b56603c5bb541fbb507" + integrity sha512-sOxsZOq25mT1wRsfHcbtkInS+Ek7Q8jCHUB0ZUTP0tc/c41QHriU/NunqMfCUWsL4H3MHpvQD4QR9kSYhS7UvQ== + dependencies: + "@jest/types" "^26.6.2" + chalk "^4.0.0" + graceful-fs "^4.2.4" + jest-pnp-resolver "^1.2.2" + jest-util "^26.6.2" + read-pkg-up "^7.0.1" + resolve "^1.18.1" + slash "^3.0.0" + +jest-runner@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-26.6.3.tgz#2d1fed3d46e10f233fd1dbd3bfaa3fe8924be159" + integrity sha512-atgKpRHnaA2OvByG/HpGA4g6CSPS/1LK0jK3gATJAoptC1ojltpmVlYC3TYgdmGp+GLuhzpH30Gvs36szSL2JQ== + dependencies: + "@jest/console" "^26.6.2" + "@jest/environment" "^26.6.2" + "@jest/test-result" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + chalk "^4.0.0" + emittery "^0.7.1" + exit "^0.1.2" + graceful-fs "^4.2.4" + jest-config "^26.6.3" + jest-docblock "^26.0.0" + jest-haste-map "^26.6.2" + jest-leak-detector "^26.6.2" + jest-message-util "^26.6.2" + jest-resolve "^26.6.2" + jest-runtime "^26.6.3" + jest-util "^26.6.2" + jest-worker "^26.6.2" + source-map-support "^0.5.6" + throat "^5.0.0" + +jest-runtime@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-26.6.3.tgz#4f64efbcfac398331b74b4b3c82d27d401b8fa2b" + integrity sha512-lrzyR3N8sacTAMeonbqpnSka1dHNux2uk0qqDXVkMv2c/A3wYnvQ4EXuI013Y6+gSKSCxdaczvf4HF0mVXHRdw== + dependencies: + "@jest/console" "^26.6.2" + "@jest/environment" "^26.6.2" + "@jest/fake-timers" "^26.6.2" + "@jest/globals" "^26.6.2" + "@jest/source-map" "^26.6.2" + "@jest/test-result" "^26.6.2" + "@jest/transform" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/yargs" "^15.0.0" + chalk "^4.0.0" + cjs-module-lexer "^0.6.0" + collect-v8-coverage "^1.0.0" + exit "^0.1.2" + glob "^7.1.3" + graceful-fs "^4.2.4" + jest-config "^26.6.3" + jest-haste-map "^26.6.2" + jest-message-util "^26.6.2" + jest-mock "^26.6.2" + jest-regex-util "^26.0.0" + jest-resolve "^26.6.2" + jest-snapshot "^26.6.2" + jest-util "^26.6.2" + jest-validate "^26.6.2" + slash "^3.0.0" + strip-bom "^4.0.0" + yargs "^15.4.1" + +jest-serializer@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-serializer/-/jest-serializer-26.6.2.tgz#d139aafd46957d3a448f3a6cdabe2919ba0742d1" + integrity sha512-S5wqyz0DXnNJPd/xfIzZ5Xnp1HrJWBczg8mMfMpN78OJ5eDxXyf+Ygld9wX1DnUWbIbhM1YDY95NjR4CBXkb2g== + dependencies: + "@types/node" "*" + graceful-fs "^4.2.4" + +jest-snapshot@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-26.6.2.tgz#f3b0af1acb223316850bd14e1beea9837fb39c84" + integrity sha512-OLhxz05EzUtsAmOMzuupt1lHYXCNib0ECyuZ/PZOx9TrZcC8vL0x+DUG3TL+GLX3yHG45e6YGjIm0XwDc3q3og== + dependencies: + "@babel/types" "^7.0.0" + "@jest/types" "^26.6.2" + "@types/babel__traverse" "^7.0.4" + "@types/prettier" "^2.0.0" + chalk "^4.0.0" + expect "^26.6.2" + graceful-fs "^4.2.4" + jest-diff "^26.6.2" + jest-get-type "^26.3.0" + jest-haste-map "^26.6.2" + jest-matcher-utils "^26.6.2" + jest-message-util "^26.6.2" + jest-resolve "^26.6.2" + natural-compare "^1.4.0" + pretty-format "^26.6.2" + semver "^7.3.2" + +jest-util@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-26.6.2.tgz#907535dbe4d5a6cb4c47ac9b926f6af29576cbc1" + integrity sha512-MDW0fKfsn0OI7MS7Euz6h8HNDXVQ0gaM9uW6RjfDmd1DAFcaxX9OqIakHIqhbnmF08Cf2DLDG+ulq8YQQ0Lp0Q== + dependencies: + "@jest/types" "^26.6.2" + "@types/node" "*" + chalk "^4.0.0" + graceful-fs "^4.2.4" + is-ci "^2.0.0" + micromatch "^4.0.2" + +jest-validate@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-26.6.2.tgz#23d380971587150467342911c3d7b4ac57ab20ec" + integrity sha512-NEYZ9Aeyj0i5rQqbq+tpIOom0YS1u2MVu6+euBsvpgIme+FOfRmoC4R5p0JiAUpaFvFy24xgrpMknarR/93XjQ== + dependencies: + "@jest/types" "^26.6.2" + camelcase "^6.0.0" + chalk "^4.0.0" + jest-get-type "^26.3.0" + leven "^3.1.0" + pretty-format "^26.6.2" + +jest-watcher@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-watcher/-/jest-watcher-26.6.2.tgz#a5b683b8f9d68dbcb1d7dae32172d2cca0592975" + integrity sha512-WKJob0P/Em2csiVthsI68p6aGKTIcsfjH9Gsx1f0A3Italz43e3ho0geSAVsmj09RWOELP1AZ/DXyJgOgDKxXQ== + dependencies: + "@jest/test-result" "^26.6.2" + "@jest/types" "^26.6.2" + "@types/node" "*" + ansi-escapes "^4.2.1" + chalk "^4.0.0" + jest-util "^26.6.2" + string-length "^4.0.1" + +jest-worker@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-26.6.2.tgz#7f72cbc4d643c365e27b9fd775f9d0eaa9c7a8ed" + integrity sha512-KWYVV1c4i+jbMpaBC+U++4Va0cp8OisU185o73T1vo99hqi7w8tSJfUXYswwqqrjzwxa6KpRK54WhPvwf5w6PQ== + dependencies: + "@types/node" "*" + merge-stream "^2.0.0" + supports-color "^7.0.0" + +jest@^26.6.3: + version "26.6.3" + resolved "https://registry.yarnpkg.com/jest/-/jest-26.6.3.tgz#40e8fdbe48f00dfa1f0ce8121ca74b88ac9148ef" + integrity sha512-lGS5PXGAzR4RF7V5+XObhqz2KZIDUA1yD0DG6pBVmy10eh0ZIXQImRuzocsI/N2XZ1GrLFwTS27In2i2jlpq1Q== + dependencies: + "@jest/core" "^26.6.3" + import-local "^3.0.2" + jest-cli "^26.6.3" + +js-tokens@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" + integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== + +js-yaml@^3.13.1: + version "3.14.1" + resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-3.14.1.tgz#dae812fdb3825fa306609a8717383c50c36a0537" + integrity sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g== + dependencies: + argparse "^1.0.7" + esprima "^4.0.0" + +jsdom@^16.4.0: + version "16.7.0" + resolved "https://registry.yarnpkg.com/jsdom/-/jsdom-16.7.0.tgz#918ae71965424b197c819f8183a754e18977b710" + integrity sha512-u9Smc2G1USStM+s/x1ru5Sxrl6mPYCbByG1U/hUmqaVsm4tbNyS7CicOSRyuGQYZhTu0h84qkZZQ/I+dzizSVw== + dependencies: + abab "^2.0.5" + acorn "^8.2.4" + acorn-globals "^6.0.0" + cssom "^0.4.4" + cssstyle "^2.3.0" + data-urls "^2.0.0" + decimal.js "^10.2.1" + domexception "^2.0.1" + escodegen "^2.0.0" + form-data "^3.0.0" + html-encoding-sniffer "^2.0.1" + http-proxy-agent "^4.0.1" + https-proxy-agent "^5.0.0" + is-potential-custom-element-name "^1.0.1" + nwsapi "^2.2.0" + parse5 "6.0.1" + saxes "^5.0.1" + symbol-tree "^3.2.4" + tough-cookie "^4.0.0" + w3c-hr-time "^1.0.2" + w3c-xmlserializer "^2.0.0" + webidl-conversions "^6.1.0" + whatwg-encoding "^1.0.5" + whatwg-mimetype "^2.3.0" + whatwg-url "^8.5.0" + ws "^7.4.6" + xml-name-validator "^3.0.0" + +jsesc@^2.5.1: + version "2.5.2" + resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-2.5.2.tgz#80564d2e483dacf6e8ef209650a67df3f0c283a4" + integrity sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA== + +jsesc@~0.5.0: + version "0.5.0" + resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-0.5.0.tgz#e7dee66e35d6fc16f710fe91d5cf69f70f08911d" + integrity sha512-uZz5UnB7u4T9LvwmFqXii7pZSouaRPorGs5who1Ip7VO0wxanFvBL7GkM6dTHlgX+jhBApRetaWpnDabOeTcnA== + +json-parse-even-better-errors@^2.3.0: + version "2.3.1" + resolved "https://registry.yarnpkg.com/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz#7c47805a94319928e05777405dc12e1f7a4ee02d" + integrity sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w== + +json5@^2.2.1: + version "2.2.1" + resolved "https://registry.yarnpkg.com/json5/-/json5-2.2.1.tgz#655d50ed1e6f95ad1a3caababd2b0efda10b395c" + integrity sha512-1hqLFMSrGHRHxav9q9gNjJ5EXznIxGVO09xQRrwplcS8qs28pZ8s8hupZAmqDwZUmVZ2Qb2jnyPOWcDH8m8dlA== + +kind-of@^3.0.2, kind-of@^3.0.3, kind-of@^3.2.0: + version "3.2.2" + resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-3.2.2.tgz#31ea21a734bab9bbb0f32466d893aea51e4a3c64" + integrity sha512-NOW9QQXMoZGg/oqnVNoNTTIFEIid1627WCffUBJEdMxYApq7mNE7CpzucIPc+ZQg25Phej7IJSmX3hO+oblOtQ== + dependencies: + is-buffer "^1.1.5" + +kind-of@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-4.0.0.tgz#20813df3d712928b207378691a45066fae72dd57" + integrity sha512-24XsCxmEbRwEDbz/qz3stgin8TTzZ1ESR56OMCN0ujYg+vRutNSiOj9bHH9u85DKgXguraugV5sFuvbD4FW/hw== + dependencies: + is-buffer "^1.1.5" + +kind-of@^5.0.0: + version "5.1.0" + resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-5.1.0.tgz#729c91e2d857b7a419a1f9aa65685c4c33f5845d" + integrity sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw== + +kind-of@^6.0.0, kind-of@^6.0.2: + version "6.0.3" + resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-6.0.3.tgz#07c05034a6c349fa06e24fa35aa76db4580ce4dd" + integrity sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw== + +kleur@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/kleur/-/kleur-3.0.3.tgz#a79c9ecc86ee1ce3fa6206d1216c501f147fc07e" + integrity sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w== + +leven@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/leven/-/leven-3.1.0.tgz#77891de834064cccba82ae7842bb6b14a13ed7f2" + integrity sha512-qsda+H8jTaUaN/x5vzW2rzc+8Rw4TAQ/4KjB46IwK5VH+IlVeeeje/EoZRpiXvIqjFgK84QffqPztGI3VBLG1A== + +levn@~0.3.0: + version "0.3.0" + resolved "https://registry.yarnpkg.com/levn/-/levn-0.3.0.tgz#3b09924edf9f083c0490fdd4c0bc4421e04764ee" + integrity sha512-0OO4y2iOHix2W6ujICbKIaEQXvFQHue65vUG3pb5EUomzPI90z9hsA1VsO/dbIIpC53J8gxM9Q4Oho0jrCM/yA== + dependencies: + prelude-ls "~1.1.2" + type-check "~0.3.2" + +lines-and-columns@^1.1.6: + version "1.2.4" + resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.2.4.tgz#eca284f75d2965079309dc0ad9255abb2ebc1632" + integrity sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg== + +locate-path@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" + integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== + dependencies: + p-locate "^4.1.0" + +lodash.debounce@^4.0.8: + version "4.0.8" + resolved "https://registry.yarnpkg.com/lodash.debounce/-/lodash.debounce-4.0.8.tgz#82d79bff30a67c4005ffd5e2515300ad9ca4d7af" + integrity sha512-FT1yDzDYEoYWhnSGnpE/4Kj1fLZkDFyqRb7fNt6FdYOSxlUWAtp42Eh6Wb0rGIv/m9Bgo7x4GhQbm5Ys4SG5ow== + +lodash@^4.7.0: + version "4.17.21" + resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" + integrity sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg== + +lru-cache@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-6.0.0.tgz#6d6fe6570ebd96aaf90fcad1dafa3b2566db3a94" + integrity sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA== + dependencies: + yallist "^4.0.0" + +make-dir@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-2.1.0.tgz#5f0310e18b8be898cc07009295a30ae41e91e6f5" + integrity sha512-LS9X+dc8KLxXCb8dni79fLIIUA5VyZoyjSMCwTluaXA0o27cCK0bhXkpgw+sTXVpPy/lSO57ilRixqk0vDmtRA== + dependencies: + pify "^4.0.1" + semver "^5.6.0" + +make-dir@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" + integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== + dependencies: + semver "^6.0.0" + +makeerror@1.0.12: + version "1.0.12" + resolved "https://registry.yarnpkg.com/makeerror/-/makeerror-1.0.12.tgz#3e5dd2079a82e812e983cc6610c4a2cb0eaa801a" + integrity sha512-JmqCvUhmt43madlpFzG4BQzG2Z3m6tvQDNKdClZnO3VbIudJYmxsT0FNJMeiB2+JTSlTQTSbU8QdesVmwJcmLg== + dependencies: + tmpl "1.0.5" + +map-cache@^0.2.2: + version "0.2.2" + resolved "https://registry.yarnpkg.com/map-cache/-/map-cache-0.2.2.tgz#c32abd0bd6525d9b051645bb4f26ac5dc98a0dbf" + integrity sha512-8y/eV9QQZCiyn1SprXSrCmqJN0yNRATe+PO8ztwqrvrbdRLA3eYJF0yaR0YayLWkMbsQSKWS9N2gPcGEc4UsZg== + +map-visit@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/map-visit/-/map-visit-1.0.0.tgz#ecdca8f13144e660f1b5bd41f12f3479d98dfb8f" + integrity sha512-4y7uGv8bd2WdM9vpQsiQNo41Ln1NvhvDRuVt0k2JZQ+ezN2uaQes7lZeZ+QQUHOLQAtDaBJ+7wCbi+ab/KFs+w== + dependencies: + object-visit "^1.0.0" + +merge-stream@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" + integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== + +micromatch@^3.1.4: + version "3.1.10" + resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-3.1.10.tgz#70859bc95c9840952f359a068a3fc49f9ecfac23" + integrity sha512-MWikgl9n9M3w+bpsY3He8L+w9eF9338xRl8IAO5viDizwSzziFEyUzo2xrrloB64ADbTf8uA8vRqqttDTOmccg== + dependencies: + arr-diff "^4.0.0" + array-unique "^0.3.2" + braces "^2.3.1" + define-property "^2.0.2" + extend-shallow "^3.0.2" + extglob "^2.0.4" + fragment-cache "^0.2.1" + kind-of "^6.0.2" + nanomatch "^1.2.9" + object.pick "^1.3.0" + regex-not "^1.0.0" + snapdragon "^0.8.1" + to-regex "^3.0.2" + +micromatch@^4.0.2: + version "4.0.5" + resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-4.0.5.tgz#bc8999a7cbbf77cdc89f132f6e467051b49090c6" + integrity sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA== + dependencies: + braces "^3.0.2" + picomatch "^2.3.1" + +mime-db@1.52.0: + version "1.52.0" + resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70" + integrity sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg== + +mime-types@^2.1.12: + version "2.1.35" + resolved "https://registry.yarnpkg.com/mime-types/-/mime-types-2.1.35.tgz#381a871b62a734450660ae3deee44813f70d959a" + integrity sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw== + dependencies: + mime-db "1.52.0" + +mimic-fn@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" + integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== + +minimatch@^3.0.4, minimatch@^3.1.1: + version "3.1.2" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" + integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw== + dependencies: + brace-expansion "^1.1.7" + +minimist@^1.1.1, minimist@^1.2.0: + version "1.2.6" + resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44" + integrity sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q== + +mixin-deep@^1.2.0: + version "1.3.2" + resolved "https://registry.yarnpkg.com/mixin-deep/-/mixin-deep-1.3.2.tgz#1120b43dc359a785dce65b55b82e257ccf479566" + integrity sha512-WRoDn//mXBiJ1H40rqa3vH0toePwSsGb45iInWlTySa+Uu4k3tYUSxa2v1KqAiLtvlrSzaExqS1gtk96A9zvEA== + dependencies: + for-in "^1.0.2" + is-extendable "^1.0.1" + +ms@2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.0.0.tgz#5608aeadfc00be6c2901df5f9861788de0d597c8" + integrity sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A== + +ms@2.1.2: + version "2.1.2" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.2.tgz#d09d1f357b443f493382a8eb3ccd183872ae6009" + integrity sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w== + +nanomatch@^1.2.9: + version "1.2.13" + resolved "https://registry.yarnpkg.com/nanomatch/-/nanomatch-1.2.13.tgz#b87a8aa4fc0de8fe6be88895b38983ff265bd119" + integrity sha512-fpoe2T0RbHwBTBUOftAfBPaDEi06ufaUai0mE6Yn1kacc3SnTErfb/h+X94VXzI64rKFHYImXSvdwGGCmwOqCA== + dependencies: + arr-diff "^4.0.0" + array-unique "^0.3.2" + define-property "^2.0.2" + extend-shallow "^3.0.2" + fragment-cache "^0.2.1" + is-windows "^1.0.2" + kind-of "^6.0.2" + object.pick "^1.3.0" + regex-not "^1.0.0" + snapdragon "^0.8.1" + to-regex "^3.0.1" + +natural-compare@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7" + integrity sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw== + +nice-try@^1.0.4: + version "1.0.5" + resolved "https://registry.yarnpkg.com/nice-try/-/nice-try-1.0.5.tgz#a3378a7696ce7d223e88fc9b764bd7ef1089e366" + integrity sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ== + +node-int64@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/node-int64/-/node-int64-0.4.0.tgz#87a9065cdb355d3182d8f94ce11188b825c68a3b" + integrity sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw== + +node-notifier@^8.0.0: + version "8.0.2" + resolved "https://registry.yarnpkg.com/node-notifier/-/node-notifier-8.0.2.tgz#f3167a38ef0d2c8a866a83e318c1ba0efeb702c5" + integrity sha512-oJP/9NAdd9+x2Q+rfphB2RJCHjod70RcRLjosiPMMu5gjIfwVnOUGq2nbTjTUbmy0DJ/tFIVT30+Qe3nzl4TJg== + dependencies: + growly "^1.3.0" + is-wsl "^2.2.0" + semver "^7.3.2" + shellwords "^0.1.1" + uuid "^8.3.0" + which "^2.0.2" + +node-releases@^2.0.5: + version "2.0.5" + resolved "https://registry.yarnpkg.com/node-releases/-/node-releases-2.0.5.tgz#280ed5bc3eba0d96ce44897d8aee478bfb3d9666" + integrity sha512-U9h1NLROZTq9uE1SNffn6WuPDg8icmi3ns4rEl/oTfIle4iLjTliCzgTsbaIFMq/Xn078/lfY/BL0GWZ+psK4Q== + +normalize-package-data@^2.5.0: + version "2.5.0" + resolved "https://registry.yarnpkg.com/normalize-package-data/-/normalize-package-data-2.5.0.tgz#e66db1838b200c1dfc233225d12cb36520e234a8" + integrity sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA== + dependencies: + hosted-git-info "^2.1.4" + resolve "^1.10.0" + semver "2 || 3 || 4 || 5" + validate-npm-package-license "^3.0.1" + +normalize-path@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-2.1.1.tgz#1ab28b556e198363a8c1a6f7e6fa20137fe6aed9" + integrity sha512-3pKJwH184Xo/lnH6oyP1q2pMd7HcypqqmRs91/6/i2CGtWwIKGCkOOMTm/zXbgTEWHw1uNpNi/igc3ePOYHb6w== + dependencies: + remove-trailing-separator "^1.0.1" + +normalize-path@^3.0.0, normalize-path@~3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" + integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== + +npm-run-path@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-2.0.2.tgz#35a9232dfa35d7067b4cb2ddf2357b1871536c5f" + integrity sha512-lJxZYlT4DW/bRUtFh1MQIWqmLwQfAxnqWG4HhEdjMlkrJYnJn0Jrr2u3mgxqaWsdiBc76TYkTG/mhrnYTuzfHw== + dependencies: + path-key "^2.0.0" + +npm-run-path@^4.0.0: + version "4.0.1" + resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-4.0.1.tgz#b7ecd1e5ed53da8e37a55e1c2269e0b97ed748ea" + integrity sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw== + dependencies: + path-key "^3.0.0" + +nwsapi@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/nwsapi/-/nwsapi-2.2.0.tgz#204879a9e3d068ff2a55139c2c772780681a38b7" + integrity sha512-h2AatdwYH+JHiZpv7pt/gSX1XoRGb7L/qSIeuqA6GwYoF9w1vP1cw42TO0aI2pNyshRK5893hNSl+1//vHK7hQ== + +object-copy@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/object-copy/-/object-copy-0.1.0.tgz#7e7d858b781bd7c991a41ba975ed3812754e998c" + integrity sha512-79LYn6VAb63zgtmAteVOWo9Vdj71ZVBy3Pbse+VqxDpEP83XuujMrGqHIwAXJ5I/aM0zU7dIyIAhifVTPrNItQ== + dependencies: + copy-descriptor "^0.1.0" + define-property "^0.2.5" + kind-of "^3.0.3" + +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-visit@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/object-visit/-/object-visit-1.0.1.tgz#f79c4493af0c5377b59fe39d395e41042dd045bb" + integrity sha512-GBaMwwAVK9qbQN3Scdo0OyvgPW7l3lnaVMj84uTOZlswkX0KpF6fyDBJhtTthf7pymztoN36/KEr1DyhF96zEA== + dependencies: + isobject "^3.0.0" + +object.assign@^4.1.0: + version "4.1.2" + resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.2.tgz#0ed54a342eceb37b38ff76eb831a0e788cb63940" + integrity sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ== + dependencies: + call-bind "^1.0.0" + define-properties "^1.1.3" + has-symbols "^1.0.1" + object-keys "^1.1.1" + +object.pick@^1.3.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/object.pick/-/object.pick-1.3.0.tgz#87a10ac4c1694bd2e1cbf53591a66141fb5dd747" + integrity sha512-tqa/UMy/CCoYmj+H5qc07qvSL9dqcs/WZENZ1JbtWBlATP+iVOe778gE6MSijnyCnORzDuX6hU+LA4SZ09YjFQ== + dependencies: + isobject "^3.0.1" + +once@^1.3.0, once@^1.3.1, once@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1" + integrity sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w== + dependencies: + wrappy "1" + +onetime@^5.1.0: + version "5.1.2" + resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" + integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== + dependencies: + mimic-fn "^2.1.0" + +optionator@^0.8.1: + version "0.8.3" + resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.8.3.tgz#84fa1d036fe9d3c7e21d99884b601167ec8fb495" + integrity sha512-+IW9pACdk3XWmmTXG8m3upGUJst5XRGzxMRjXzAuJ1XnIFNvfhjjIuYkDvysnPQ7qzqVzLt78BCruntqRhWQbA== + dependencies: + deep-is "~0.1.3" + fast-levenshtein "~2.0.6" + levn "~0.3.0" + prelude-ls "~1.1.2" + type-check "~0.3.2" + word-wrap "~1.2.3" + +p-each-series@^2.1.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/p-each-series/-/p-each-series-2.2.0.tgz#105ab0357ce72b202a8a8b94933672657b5e2a9a" + integrity sha512-ycIL2+1V32th+8scbpTvyHNaHe02z0sjgh91XXjAk+ZeXoPN4Z46DVUnzdso0aX4KckKw0FNNFHdjZ2UsZvxiA== + +p-finally@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/p-finally/-/p-finally-1.0.0.tgz#3fbcfb15b899a44123b34b6dcc18b724336a2cae" + integrity sha512-LICb2p9CB7FS+0eR1oqWnHhp0FljGLZCWBE9aix0Uye9W8LTQPwMTYVGWQWIw9RdQiDg4+epXQODwIYJtSJaow== + +p-limit@^2.2.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" + integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== + dependencies: + p-try "^2.0.0" + +p-locate@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" + integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== + dependencies: + p-limit "^2.2.0" + +p-try@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" + integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== + +parse-json@^5.0.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/parse-json/-/parse-json-5.2.0.tgz#c76fc66dee54231c962b22bcc8a72cf2f99753cd" + integrity sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg== + dependencies: + "@babel/code-frame" "^7.0.0" + error-ex "^1.3.1" + json-parse-even-better-errors "^2.3.0" + lines-and-columns "^1.1.6" + +parse5@6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/parse5/-/parse5-6.0.1.tgz#e1a1c085c569b3dc08321184f19a39cc27f7c30b" + integrity sha512-Ofn/CTFzRGTTxwpNEs9PP93gXShHcTq255nzRYSKe8AkVpZY7e1fpmTfOyoIvjP5HG7Z2ZM7VS9PPhQGW2pOpw== + +pascalcase@^0.1.1: + version "0.1.1" + resolved "https://registry.yarnpkg.com/pascalcase/-/pascalcase-0.1.1.tgz#b363e55e8006ca6fe21784d2db22bd15d7917f14" + integrity sha512-XHXfu/yOQRy9vYOtUDVMN60OEJjW013GoObG1o+xwQTpB9eYJX/BjXMsdW13ZDPruFhYYn0AG22w0xgQMwl3Nw== + +path-exists@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-4.0.0.tgz#513bdbe2d3b95d7762e8c1137efa195c6c61b5b3" + integrity sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w== + +path-is-absolute@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/path-is-absolute/-/path-is-absolute-1.0.1.tgz#174b9268735534ffbc7ace6bf53a5a9e1b5c5f5f" + integrity sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg== + +path-key@^2.0.0, path-key@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/path-key/-/path-key-2.0.1.tgz#411cadb574c5a140d3a4b1910d40d80cc9f40b40" + integrity sha512-fEHGKCSmUSDPv4uoj8AlD+joPlq3peND+HRYyxFz4KPw4z926S/b8rIuFs2FYJg3BwsxJf6A9/3eIdLaYC+9Dw== + +path-key@^3.0.0, path-key@^3.1.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" + integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== + +path-parse@^1.0.7: + version "1.0.7" + resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" + integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== + +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + +picomatch@^2.0.4, picomatch@^2.2.1, picomatch@^2.3.1: + version "2.3.1" + resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" + integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== + +pify@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/pify/-/pify-4.0.1.tgz#4b2cd25c50d598735c50292224fd8c6df41e3231" + integrity sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g== + +pirates@^4.0.1: + version "4.0.5" + resolved "https://registry.yarnpkg.com/pirates/-/pirates-4.0.5.tgz#feec352ea5c3268fb23a37c702ab1699f35a5f3b" + integrity sha512-8V9+HQPupnaXMA23c5hvl69zXvTwTzyAYasnkb0Tts4XvO4CliqONMOnvlq26rkhLC3nWDFBJf73LU1e1VZLaQ== + +pkg-dir@^4.2.0: + version "4.2.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" + integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== + dependencies: + find-up "^4.0.0" + +posix-character-classes@^0.1.0: + version "0.1.1" + resolved "https://registry.yarnpkg.com/posix-character-classes/-/posix-character-classes-0.1.1.tgz#01eac0fe3b5af71a2a6c02feabb8c1fef7e00eab" + integrity sha512-xTgYBc3fuo7Yt7JbiuFxSYGToMoz8fLoE6TC9Wx1P/u+LfeThMOAqmuyECnlBaaJb+u1m9hHiXUEtwW4OzfUJg== + +prelude-ls@~1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.1.2.tgz#21932a549f5e52ffd9a827f570e04be62a97da54" + integrity sha512-ESF23V4SKG6lVSGZgYNpbsiaAkdab6ZgOxe52p7+Kid3W3u3bxR4Vfd/o21dmN7jSt0IwgZ4v5MUd26FEtXE9w== + +prettier@^2.2.1: + version "2.6.2" + resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.6.2.tgz#e26d71a18a74c3d0f0597f55f01fb6c06c206032" + integrity sha512-PkUpF+qoXTqhOeWL9fu7As8LXsIUZ1WYaJiY/a7McAQzxjk82OF0tibkFXVCDImZtWxbvojFjerkiLb0/q8mew== + +pretty-format@^26.6.2: + version "26.6.2" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-26.6.2.tgz#e35c2705f14cb7fe2fe94fa078345b444120fc93" + integrity sha512-7AeGuCYNGmycyQbCqd/3PWH4eOoX/OiCa0uphp57NVTeAGdJGaAliecxwBDHYQCIvrW7aDBZCYeNTP/WX69mkg== + dependencies: + "@jest/types" "^26.6.2" + ansi-regex "^5.0.0" + ansi-styles "^4.0.0" + react-is "^17.0.1" + +prompts@^2.0.1: + version "2.4.2" + resolved "https://registry.yarnpkg.com/prompts/-/prompts-2.4.2.tgz#7b57e73b3a48029ad10ebd44f74b01722a4cb069" + integrity sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q== + dependencies: + kleur "^3.0.3" + sisteransi "^1.0.5" + +psl@^1.1.33: + version "1.8.0" + resolved "https://registry.yarnpkg.com/psl/-/psl-1.8.0.tgz#9326f8bcfb013adcc005fdff056acce020e51c24" + integrity sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ== + +pump@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/pump/-/pump-3.0.0.tgz#b4a2116815bde2f4e1ea602354e8c75565107a64" + integrity sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww== + dependencies: + end-of-stream "^1.1.0" + once "^1.3.1" + +punycode@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec" + integrity sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A== + +react-is@^17.0.1: + version "17.0.2" + resolved "https://registry.yarnpkg.com/react-is/-/react-is-17.0.2.tgz#e691d4a8e9c789365655539ab372762b0efb54f0" + integrity sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w== + +read-pkg-up@^7.0.1: + version "7.0.1" + resolved "https://registry.yarnpkg.com/read-pkg-up/-/read-pkg-up-7.0.1.tgz#f3a6135758459733ae2b95638056e1854e7ef507" + integrity sha512-zK0TB7Xd6JpCLmlLmufqykGE+/TlOePD6qKClNW7hHDKFh/J7/7gCWGR7joEQEW1bKq3a3yUZSObOoWLFQ4ohg== + dependencies: + find-up "^4.1.0" + read-pkg "^5.2.0" + type-fest "^0.8.1" + +read-pkg@^5.2.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/read-pkg/-/read-pkg-5.2.0.tgz#7bf295438ca5a33e56cd30e053b34ee7250c93cc" + integrity sha512-Ug69mNOpfvKDAc2Q8DRpMjjzdtrnv9HcSMX+4VsZxD1aZ6ZzrIE7rlzXBtWTyhULSMKg076AW6WR5iZpD0JiOg== + dependencies: + "@types/normalize-package-data" "^2.4.0" + normalize-package-data "^2.5.0" + parse-json "^5.0.0" + type-fest "^0.6.0" + +readdirp@~3.6.0: + version "3.6.0" + resolved "https://registry.yarnpkg.com/readdirp/-/readdirp-3.6.0.tgz#74a370bd857116e245b29cc97340cd431a02a6c7" + integrity sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA== + dependencies: + picomatch "^2.2.1" + +regenerate-unicode-properties@^10.0.1: + version "10.0.1" + resolved "https://registry.yarnpkg.com/regenerate-unicode-properties/-/regenerate-unicode-properties-10.0.1.tgz#7f442732aa7934a3740c779bb9b3340dccc1fb56" + integrity sha512-vn5DU6yg6h8hP/2OkQo3K7uVILvY4iu0oI4t3HFa81UPkhGJwkRwM10JEc3upjdhHjs/k8GJY1sRBhk5sr69Bw== + dependencies: + regenerate "^1.4.2" + +regenerate@^1.4.2: + version "1.4.2" + resolved "https://registry.yarnpkg.com/regenerate/-/regenerate-1.4.2.tgz#b9346d8827e8f5a32f7ba29637d398b69014848a" + integrity sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A== + +regenerator-runtime@^0.13.4: + version "0.13.9" + resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz#8925742a98ffd90814988d7566ad30ca3b263b52" + integrity sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA== + +regenerator-transform@^0.15.0: + version "0.15.0" + resolved "https://registry.yarnpkg.com/regenerator-transform/-/regenerator-transform-0.15.0.tgz#cbd9ead5d77fae1a48d957cf889ad0586adb6537" + integrity sha512-LsrGtPmbYg19bcPHwdtmXwbW+TqNvtY4riE3P83foeHRroMbH6/2ddFBfab3t7kbzc7v7p4wbkIecHImqt0QNg== + dependencies: + "@babel/runtime" "^7.8.4" + +regex-not@^1.0.0, regex-not@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/regex-not/-/regex-not-1.0.2.tgz#1f4ece27e00b0b65e0247a6810e6a85d83a5752c" + integrity sha512-J6SDjUgDxQj5NusnOtdFxDwN/+HWykR8GELwctJ7mdqhcyy1xEc4SRFHUXvxTp661YaVKAjfRLZ9cCqS6tn32A== + dependencies: + extend-shallow "^3.0.2" + safe-regex "^1.1.0" + +regexpu-core@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/regexpu-core/-/regexpu-core-5.0.1.tgz#c531122a7840de743dcf9c83e923b5560323ced3" + integrity sha512-CriEZlrKK9VJw/xQGJpQM5rY88BtuL8DM+AEwvcThHilbxiTAy8vq4iJnd2tqq8wLmjbGZzP7ZcKFjbGkmEFrw== + dependencies: + regenerate "^1.4.2" + regenerate-unicode-properties "^10.0.1" + regjsgen "^0.6.0" + regjsparser "^0.8.2" + unicode-match-property-ecmascript "^2.0.0" + unicode-match-property-value-ecmascript "^2.0.0" + +regjsgen@^0.6.0: + version "0.6.0" + resolved "https://registry.yarnpkg.com/regjsgen/-/regjsgen-0.6.0.tgz#83414c5354afd7d6627b16af5f10f41c4e71808d" + integrity sha512-ozE883Uigtqj3bx7OhL1KNbCzGyW2NQZPl6Hs09WTvCuZD5sTI4JY58bkbQWa/Y9hxIsvJ3M8Nbf7j54IqeZbA== + +regjsparser@^0.8.2: + version "0.8.4" + resolved "https://registry.yarnpkg.com/regjsparser/-/regjsparser-0.8.4.tgz#8a14285ffcc5de78c5b95d62bbf413b6bc132d5f" + integrity sha512-J3LABycON/VNEu3abOviqGHuB/LOtOQj8SKmfP9anY5GfAVw/SPjwzSjxGjbZXIxbGfqTHtJw58C2Li/WkStmA== + dependencies: + jsesc "~0.5.0" + +remove-trailing-separator@^1.0.1: + version "1.1.0" + resolved "https://registry.yarnpkg.com/remove-trailing-separator/-/remove-trailing-separator-1.1.0.tgz#c24bce2a283adad5bc3f58e0d48249b92379d8ef" + integrity sha512-/hS+Y0u3aOfIETiaiirUFwDBDzmXPvO+jAfKTitUngIPzdKc6Z0LoFjM/CK5PL4C+eKwHohlHAb6H0VFfmmUsw== + +repeat-element@^1.1.2: + version "1.1.4" + resolved "https://registry.yarnpkg.com/repeat-element/-/repeat-element-1.1.4.tgz#be681520847ab58c7568ac75fbfad28ed42d39e9" + integrity sha512-LFiNfRcSu7KK3evMyYOuCzv3L10TW7yC1G2/+StMjK8Y6Vqd2MG7r/Qjw4ghtuCOjFvlnms/iMmLqpvW/ES/WQ== + +repeat-string@^1.6.1: + version "1.6.1" + resolved "https://registry.yarnpkg.com/repeat-string/-/repeat-string-1.6.1.tgz#8dcae470e1c88abc2d600fff4a776286da75e637" + integrity sha512-PV0dzCYDNfRi1jCDbJzpW7jNNDRuCOG/jI5ctQcGKt/clZD+YcPS3yIlWuTJMmESC8aevCFmWJy5wjAFgNqN6w== + +require-directory@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" + integrity sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q== + +require-main-filename@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/require-main-filename/-/require-main-filename-2.0.0.tgz#d0b329ecc7cc0f61649f62215be69af54aa8989b" + integrity sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg== + +resolve-cwd@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/resolve-cwd/-/resolve-cwd-3.0.0.tgz#0f0075f1bb2544766cf73ba6a6e2adfebcb13f2d" + integrity sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg== + dependencies: + resolve-from "^5.0.0" + +resolve-from@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-5.0.0.tgz#c35225843df8f776df21c57557bc087e9dfdfc69" + integrity sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw== + +resolve-url@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/resolve-url/-/resolve-url-0.2.1.tgz#2c637fe77c893afd2a663fe21aa9080068e2052a" + integrity sha512-ZuF55hVUQaaczgOIwqWzkEcEidmlD/xl44x1UZnhOXcYuFN2S6+rcxpG+C1N3So0wvNI3DmJICUFfu2SxhBmvg== + +resolve@^1.10.0, resolve@^1.14.2, resolve@^1.18.1: + version "1.22.0" + resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.0.tgz#5e0b8c67c15df57a89bdbabe603a002f21731198" + integrity sha512-Hhtrw0nLeSrFQ7phPp4OOcVjLPIeMnRlr5mcnVuMe7M/7eBn98A3hmFRLoFo3DLZkivSYwhRUJTyPyWAk56WLw== + dependencies: + is-core-module "^2.8.1" + path-parse "^1.0.7" + supports-preserve-symlinks-flag "^1.0.0" + +ret@~0.1.10: + version "0.1.15" + resolved "https://registry.yarnpkg.com/ret/-/ret-0.1.15.tgz#b8a4825d5bdb1fc3f6f53c2bc33f81388681c7bc" + integrity sha512-TTlYpa+OL+vMMNG24xSlQGEJ3B/RzEfUlLct7b5G/ytav+wPrplCpVMFuwzXbkecJrb6IYo1iFb0S9v37754mg== + +rimraf@^3.0.0: + version "3.0.2" + resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-3.0.2.tgz#f1a5402ba6220ad52cc1282bac1ae3aa49fd061a" + integrity sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA== + dependencies: + glob "^7.1.3" + +rsvp@^4.8.4: + version "4.8.5" + resolved "https://registry.yarnpkg.com/rsvp/-/rsvp-4.8.5.tgz#c8f155311d167f68f21e168df71ec5b083113734" + integrity sha512-nfMOlASu9OnRJo1mbEk2cz0D56a1MBNrJ7orjRZQG10XDyuvwksKbuXNp6qa+kbn839HwjwhBzhFmdsaEAfauA== + +safe-buffer@~5.1.1: + version "5.1.2" + resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" + integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== + +safe-regex@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/safe-regex/-/safe-regex-1.1.0.tgz#40a3669f3b077d1e943d44629e157dd48023bf2e" + integrity sha512-aJXcif4xnaNUzvUuC5gcb46oTS7zvg4jpMTnuqtrEPlR3vFr4pxtdTwaF1Qs3Enjn9HK+ZlwQui+a7z0SywIzg== + dependencies: + ret "~0.1.10" + +"safer-buffer@>= 2.1.2 < 3": + version "2.1.2" + resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" + integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== + +sane@^4.0.3: + version "4.1.0" + resolved "https://registry.yarnpkg.com/sane/-/sane-4.1.0.tgz#ed881fd922733a6c461bc189dc2b6c006f3ffded" + integrity sha512-hhbzAgTIX8O7SHfp2c8/kREfEn4qO/9q8C9beyY6+tvZ87EpoZ3i1RIEvp27YBswnNbY9mWd6paKVmKbAgLfZA== + dependencies: + "@cnakazawa/watch" "^1.0.3" + anymatch "^2.0.0" + capture-exit "^2.0.0" + exec-sh "^0.3.2" + execa "^1.0.0" + fb-watchman "^2.0.0" + micromatch "^3.1.4" + minimist "^1.1.1" + walker "~1.0.5" + +saxes@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/saxes/-/saxes-5.0.1.tgz#eebab953fa3b7608dbe94e5dadb15c888fa6696d" + integrity sha512-5LBh1Tls8c9xgGjw3QrMwETmTMVk0oFgvrFSvWx62llR2hcEInrKNZ2GZCCuuy2lvWrdl5jhbpeqc5hRYKFOcw== + dependencies: + xmlchars "^2.2.0" + +"semver@2 || 3 || 4 || 5", semver@^5.5.0, semver@^5.6.0: + version "5.7.1" + resolved "https://registry.yarnpkg.com/semver/-/semver-5.7.1.tgz#a954f931aeba508d307bbf069eff0c01c96116f7" + integrity sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ== + +semver@7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.0.0.tgz#5f3ca35761e47e05b206c6daff2cf814f0316b8e" + integrity sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A== + +semver@^6.0.0, semver@^6.1.1, semver@^6.1.2, semver@^6.3.0: + version "6.3.0" + resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d" + integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw== + +semver@^7.3.2: + version "7.3.7" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.7.tgz#12c5b649afdbf9049707796e22a4028814ce523f" + integrity sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g== + dependencies: + lru-cache "^6.0.0" + +set-blocking@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/set-blocking/-/set-blocking-2.0.0.tgz#045f9782d011ae9a6803ddd382b24392b3d890f7" + integrity sha512-KiKBS8AnWGEyLzofFfmvKwpdPzqiy16LvQfK3yv/fVH7Bj13/wl3JSR1J+rfgRE9q7xUJK4qvgS8raSOeLUehw== + +set-value@^2.0.0, set-value@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/set-value/-/set-value-2.0.1.tgz#a18d40530e6f07de4228c7defe4227af8cad005b" + integrity sha512-JxHc1weCN68wRY0fhCoXpyK55m/XPHafOmK4UWD7m2CI14GMcFypt4w/0+NV5f/ZMby2F6S2wwA7fgynh9gWSw== + dependencies: + extend-shallow "^2.0.1" + is-extendable "^0.1.1" + is-plain-object "^2.0.3" + split-string "^3.0.1" + +shebang-command@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/shebang-command/-/shebang-command-1.2.0.tgz#44aac65b695b03398968c39f363fee5deafdf1ea" + integrity sha512-EV3L1+UQWGor21OmnvojK36mhg+TyIKDh3iFBKBohr5xeXIhNBcx8oWdgkTEEQ+BEFFYdLRuqMfd5L84N1V5Vg== + dependencies: + shebang-regex "^1.0.0" + +shebang-command@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/shebang-command/-/shebang-command-2.0.0.tgz#ccd0af4f8835fbdc265b82461aaf0c36663f34ea" + integrity sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA== + dependencies: + shebang-regex "^3.0.0" + +shebang-regex@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/shebang-regex/-/shebang-regex-1.0.0.tgz#da42f49740c0b42db2ca9728571cb190c98efea3" + integrity sha512-wpoSFAxys6b2a2wHZ1XpDSgD7N9iVjg29Ph9uV/uaP9Ex/KXlkTZTeddxDPSYQpgvzKLGJke2UU0AzoGCjNIvQ== + +shebang-regex@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/shebang-regex/-/shebang-regex-3.0.0.tgz#ae16f1644d873ecad843b0307b143362d4c42172" + integrity sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A== + +shellwords@^0.1.1: + version "0.1.1" + resolved "https://registry.yarnpkg.com/shellwords/-/shellwords-0.1.1.tgz#d6b9181c1a48d397324c84871efbcfc73fc0654b" + integrity sha512-vFwSUfQvqybiICwZY5+DAWIPLKsWO31Q91JSKl3UYv+K5c2QRPzn0qzec6QPu1Qc9eHYItiP3NdJqNVqetYAww== + +signal-exit@^3.0.0, signal-exit@^3.0.2: + version "3.0.7" + resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9" + integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ== + +sisteransi@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-1.0.5.tgz#134d681297756437cc05ca01370d3a7a571075ed" + integrity sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg== + +slash@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-2.0.0.tgz#de552851a1759df3a8f206535442f5ec4ddeab44" + integrity sha512-ZYKh3Wh2z1PpEXWr0MpSBZ0V6mZHAQfYevttO11c51CaWjGTaadiKZ+wVt1PbMlDV5qhMFslpZCemhwOK7C89A== + +slash@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-3.0.0.tgz#6539be870c165adbd5240220dbe361f1bc4d4634" + integrity sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q== + +snapdragon-node@^2.0.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/snapdragon-node/-/snapdragon-node-2.1.1.tgz#6c175f86ff14bdb0724563e8f3c1b021a286853b" + integrity sha512-O27l4xaMYt/RSQ5TR3vpWCAB5Kb/czIcqUFOM/C4fYcLnbZUc1PkjTAMjof2pBWaSTwOUd6qUHcFGVGj7aIwnw== + dependencies: + define-property "^1.0.0" + isobject "^3.0.0" + snapdragon-util "^3.0.1" + +snapdragon-util@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/snapdragon-util/-/snapdragon-util-3.0.1.tgz#f956479486f2acd79700693f6f7b805e45ab56e2" + integrity sha512-mbKkMdQKsjX4BAL4bRYTj21edOf8cN7XHdYUJEe+Zn99hVEYcMvKPct1IqNe7+AZPirn8BCDOQBHQZknqmKlZQ== + dependencies: + kind-of "^3.2.0" + +snapdragon@^0.8.1: + version "0.8.2" + resolved "https://registry.yarnpkg.com/snapdragon/-/snapdragon-0.8.2.tgz#64922e7c565b0e14204ba1aa7d6964278d25182d" + integrity sha512-FtyOnWN/wCHTVXOMwvSv26d+ko5vWlIDD6zoUJ7LW8vh+ZBC8QdljveRP+crNrtBwioEUWy/4dMtbBjA4ioNlg== + dependencies: + base "^0.11.1" + debug "^2.2.0" + define-property "^0.2.5" + extend-shallow "^2.0.1" + map-cache "^0.2.2" + source-map "^0.5.6" + source-map-resolve "^0.5.0" + use "^3.1.0" + +source-map-resolve@^0.5.0: + version "0.5.3" + resolved "https://registry.yarnpkg.com/source-map-resolve/-/source-map-resolve-0.5.3.tgz#190866bece7553e1f8f267a2ee82c606b5509a1a" + integrity sha512-Htz+RnsXWk5+P2slx5Jh3Q66vhQj1Cllm0zvnaY98+NFx+Dv2CF/f5O/t8x+KaNdrdIAsruNzoh/KpialbqAnw== + dependencies: + atob "^2.1.2" + decode-uri-component "^0.2.0" + resolve-url "^0.2.1" + source-map-url "^0.4.0" + urix "^0.1.0" + +source-map-support@^0.5.6: + version "0.5.21" + resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.21.tgz#04fe7c7f9e1ed2d662233c28cb2b35b9f63f6e4f" + integrity sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w== + dependencies: + buffer-from "^1.0.0" + source-map "^0.6.0" + +source-map-url@^0.4.0: + version "0.4.1" + resolved "https://registry.yarnpkg.com/source-map-url/-/source-map-url-0.4.1.tgz#0af66605a745a5a2f91cf1bbf8a7afbc283dec56" + integrity sha512-cPiFOTLUKvJFIg4SKVScy4ilPPW6rFgMgfuZJPNoDuMs3nC1HbMUycBoJw77xFIp6z1UJQJOfx6C9GMH80DiTw== + +source-map@^0.5.6: + version "0.5.7" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.5.7.tgz#8a039d2d1021d22d1ea14c80d8ea468ba2ef3fcc" + integrity sha512-LbrmJOMUSdEVxIKvdcJzQC+nQhe8FUZQTXQy6+I75skNgn3OoQ0DZA8YnFa7gp8tqtL3KPf1kmo0R5DoApeSGQ== + +source-map@^0.6.0, source-map@^0.6.1, source-map@~0.6.1: + version "0.6.1" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" + integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== + +source-map@^0.7.3: + version "0.7.4" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.7.4.tgz#a9bbe705c9d8846f4e08ff6765acf0f1b0898656" + integrity sha512-l3BikUxvPOcn5E74dZiq5BGsTb5yEwhaTSzccU6t4sDOH8NWJCstKO5QT2CvtFoK6F0saL7p9xHAqHOlCPJygA== + +spdx-correct@^3.0.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/spdx-correct/-/spdx-correct-3.1.1.tgz#dece81ac9c1e6713e5f7d1b6f17d468fa53d89a9" + integrity sha512-cOYcUWwhCuHCXi49RhFRCyJEK3iPj1Ziz9DpViV3tbZOwXD49QzIN3MpOLJNxh2qwq2lJJZaKMVw9qNi4jTC0w== + dependencies: + spdx-expression-parse "^3.0.0" + spdx-license-ids "^3.0.0" + +spdx-exceptions@^2.1.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz#3f28ce1a77a00372683eade4a433183527a2163d" + integrity sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A== + +spdx-expression-parse@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz#cf70f50482eefdc98e3ce0a6833e4a53ceeba679" + integrity sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q== + dependencies: + spdx-exceptions "^2.1.0" + spdx-license-ids "^3.0.0" + +spdx-license-ids@^3.0.0: + version "3.0.11" + resolved "https://registry.yarnpkg.com/spdx-license-ids/-/spdx-license-ids-3.0.11.tgz#50c0d8c40a14ec1bf449bae69a0ea4685a9d9f95" + integrity sha512-Ctl2BrFiM0X3MANYgj3CkygxhRmr9mi6xhejbdO960nF6EDJApTYpn0BQnDKlnNBULKiCN1n3w9EBkHK8ZWg+g== + +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" + integrity sha512-NzNVhJDYpwceVVii8/Hu6DKfD2G+NrQHlS/V/qgv763EYudVwEcMQNxd2lh+0VrUByXN/oJkl5grOhYWvQUYiw== + dependencies: + extend-shallow "^3.0.0" + +sprintf-js@~1.0.2: + version "1.0.3" + resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c" + integrity sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g== + +stack-utils@^2.0.2: + version "2.0.5" + resolved "https://registry.yarnpkg.com/stack-utils/-/stack-utils-2.0.5.tgz#d25265fca995154659dbbfba3b49254778d2fdd5" + integrity sha512-xrQcmYhOsn/1kX+Vraq+7j4oE2j/6BFscZ0etmYg81xuM8Gq0022Pxb8+IqgOFUIaxHs0KaSb7T1+OegiNrNFA== + dependencies: + escape-string-regexp "^2.0.0" + +static-extend@^0.1.1: + version "0.1.2" + resolved "https://registry.yarnpkg.com/static-extend/-/static-extend-0.1.2.tgz#60809c39cbff55337226fd5e0b520f341f1fb5c6" + integrity sha512-72E9+uLc27Mt718pMHt9VMNiAL4LMsmDbBva8mxWUCkT07fSzEGMYUCk0XWY6lp0j6RBAG4cJ3mWuZv2OE3s0g== + dependencies: + define-property "^0.2.5" + object-copy "^0.1.0" + +string-length@^4.0.1: + version "4.0.2" + resolved "https://registry.yarnpkg.com/string-length/-/string-length-4.0.2.tgz#a8a8dc7bd5c1a82b9b3c8b87e125f66871b6e57a" + integrity sha512-+l6rNN5fYHNhZZy41RXsYptCjA2Igmq4EG7kZAYFQI1E1VTXarr6ZPXBg6eq7Y6eK4FEhY6AJlyuFIb/v/S0VQ== + dependencies: + char-regex "^1.0.2" + strip-ansi "^6.0.0" + +string-width@^4.1.0, string-width@^4.2.0: + version "4.2.3" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" + integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== + dependencies: + emoji-regex "^8.0.0" + is-fullwidth-code-point "^3.0.0" + strip-ansi "^6.0.1" + +strip-ansi@^6.0.0, strip-ansi@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" + integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== + dependencies: + ansi-regex "^5.0.1" + +strip-bom@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-4.0.0.tgz#9c3505c1db45bcedca3d9cf7a16f5c5aa3901878" + integrity sha512-3xurFv5tEgii33Zi8Jtp55wEIILR9eh34FAW00PZf+JnSsTmV/ioewSgQl97JHvgjoRGwPShsWm+IdrxB35d0w== + +strip-eof@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/strip-eof/-/strip-eof-1.0.0.tgz#bb43ff5598a6eb05d89b59fcd129c983313606bf" + integrity sha512-7FCwGGmx8mD5xQd3RPUvnSpUXHM3BWuzjtpD4TXsfcZ9EL4azvVVUscFYwD9nx8Kh+uCBC00XBtAykoMHwTh8Q== + +strip-final-newline@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/strip-final-newline/-/strip-final-newline-2.0.0.tgz#89b852fb2fcbe936f6f4b3187afb0a12c1ab58ad" + integrity sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA== + +supports-color@^5.3.0: + version "5.5.0" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-5.5.0.tgz#e2e69a44ac8772f78a1ec0b35b689df6530efc8f" + integrity sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow== + dependencies: + has-flag "^3.0.0" + +supports-color@^7.0.0, supports-color@^7.1.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-7.2.0.tgz#1b7dcdcb32b8138801b3e478ba6a51caa89648da" + integrity sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw== + dependencies: + has-flag "^4.0.0" + +supports-hyperlinks@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/supports-hyperlinks/-/supports-hyperlinks-2.2.0.tgz#4f77b42488765891774b70c79babd87f9bd594bb" + integrity sha512-6sXEzV5+I5j8Bmq9/vUphGRM/RJNT9SCURJLjwfOg51heRtguGWDzcaBlgAzKhQa0EVNpPEKzQuBwZ8S8WaCeQ== + dependencies: + has-flag "^4.0.0" + supports-color "^7.0.0" + +supports-preserve-symlinks-flag@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" + integrity sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w== + +symbol-tree@^3.2.4: + version "3.2.4" + resolved "https://registry.yarnpkg.com/symbol-tree/-/symbol-tree-3.2.4.tgz#430637d248ba77e078883951fb9aa0eed7c63fa2" + integrity sha512-9QNk5KwDF+Bvz+PyObkmSYjI5ksVUYtjW7AU22r2NKcfLJcXp96hkDWU3+XndOsUb+AQ9QhfzfCT2O+CNWT5Tw== + +terminal-link@^2.0.0: + version "2.1.1" + resolved "https://registry.yarnpkg.com/terminal-link/-/terminal-link-2.1.1.tgz#14a64a27ab3c0df933ea546fba55f2d078edc994" + integrity sha512-un0FmiRUQNr5PJqy9kP7c40F5BOfpGlYTrxonDChEZB7pzZxRNp/bt+ymiy9/npwXya9KH99nJ/GXFIiUkYGFQ== + dependencies: + ansi-escapes "^4.2.1" + supports-hyperlinks "^2.0.0" + +test-exclude@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/test-exclude/-/test-exclude-6.0.0.tgz#04a8698661d805ea6fa293b6cb9e63ac044ef15e" + integrity sha512-cAGWPIyOHU6zlmg88jwm7VRyXnMN7iV68OGAbYDk/Mh/xC/pzVPlQtY6ngoIH/5/tciuhGfvESU8GrHrcxD56w== + dependencies: + "@istanbuljs/schema" "^0.1.2" + glob "^7.1.4" + minimatch "^3.0.4" + +throat@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/throat/-/throat-5.0.0.tgz#c5199235803aad18754a667d659b5e72ce16764b" + integrity sha512-fcwX4mndzpLQKBS1DVYhGAcYaYt7vsHNIvQV+WXMvnow5cgjPphq5CaayLaGsjRdSCKZFNGt7/GYAuXaNOiYCA== + +tmpl@1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/tmpl/-/tmpl-1.0.5.tgz#8683e0b902bb9c20c4f726e3c0b69f36518c07cc" + integrity sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw== + +to-fast-properties@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e" + integrity sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog== + +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" + integrity sha512-9mWHdnGRuh3onocaHzukyvCZhzvr6tiflAy/JRFXcJX0TjgfWA9pk9t8CMbzmBE4Jfw58pXbkngtBtqYxzNEyg== + dependencies: + kind-of "^3.0.2" + +to-regex-range@^2.1.0: + version "2.1.1" + resolved "https://registry.yarnpkg.com/to-regex-range/-/to-regex-range-2.1.1.tgz#7c80c17b9dfebe599e27367e0d4dd5590141db38" + integrity sha512-ZZWNfCjUokXXDGXFpZehJIkZqq91BcULFq/Pi7M5i4JnxXdhMKAK682z8bCW3o8Hj1wuuzoKcW3DfVzaP6VuNg== + dependencies: + is-number "^3.0.0" + repeat-string "^1.6.1" + +to-regex-range@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/to-regex-range/-/to-regex-range-5.0.1.tgz#1648c44aae7c8d988a326018ed72f5b4dd0392e4" + integrity sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ== + dependencies: + is-number "^7.0.0" + +to-regex@^3.0.1, to-regex@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/to-regex/-/to-regex-3.0.2.tgz#13cfdd9b336552f30b51f33a8ae1b42a7a7599ce" + integrity sha512-FWtleNAtZ/Ki2qtqej2CXTOayOH9bHDQF+Q48VpWyDXjbYxA4Yz8iDB31zXOBUlOHHKidDbqGVrTUvQMPmBGBw== + dependencies: + define-property "^2.0.2" + extend-shallow "^3.0.2" + regex-not "^1.0.2" + safe-regex "^1.1.0" + +tough-cookie@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/tough-cookie/-/tough-cookie-4.0.0.tgz#d822234eeca882f991f0f908824ad2622ddbece4" + integrity sha512-tHdtEpQCMrc1YLrMaqXXcj6AxhYi/xgit6mZu1+EDWUn+qhUf8wMQoFIy9NXuq23zAwtcB0t/MjACGR18pcRbg== + dependencies: + psl "^1.1.33" + punycode "^2.1.1" + universalify "^0.1.2" + +tr46@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/tr46/-/tr46-2.1.0.tgz#fa87aa81ca5d5941da8cbf1f9b749dc969a4e240" + integrity sha512-15Ih7phfcdP5YxqiB+iDtLoaTz4Nd35+IiAv0kQ5FNKHzXgdWqPoTIqEDDJmXceQt4JZk6lVPT8lnDlPpGDppw== + dependencies: + punycode "^2.1.1" + +type-check@~0.3.2: + version "0.3.2" + resolved "https://registry.yarnpkg.com/type-check/-/type-check-0.3.2.tgz#5884cab512cf1d355e3fb784f30804b2b520db72" + integrity sha512-ZCmOJdvOWDBYJlzAoFkC+Q0+bUyEOS1ltgp1MGU03fqHG+dbi9tBFU2Rd9QKiDZFAYrhPh2JUf7rZRIuHRKtOg== + dependencies: + prelude-ls "~1.1.2" + +type-detect@4.0.8: + version "4.0.8" + resolved "https://registry.yarnpkg.com/type-detect/-/type-detect-4.0.8.tgz#7646fb5f18871cfbb7749e69bd39a6388eb7450c" + integrity sha512-0fr/mIH1dlO+x7TlcMy+bIDqKPsw/70tVyeHW787goQjhmqaZe10uwLujubK9q9Lg6Fiho1KUKDYz0Z7k7g5/g== + +type-fest@^0.21.3: + version "0.21.3" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.21.3.tgz#d260a24b0198436e133fa26a524a6d65fa3b2e37" + integrity sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w== + +type-fest@^0.6.0: + version "0.6.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.6.0.tgz#8d2a2370d3df886eb5c90ada1c5bf6188acf838b" + integrity sha512-q+MB8nYR1KDLrgr4G5yemftpMC7/QLqVndBmEEdqzmNj5dcFOO4Oo8qlwZE3ULT3+Zim1F8Kq4cBnikNhlCMlg== + +type-fest@^0.8.1: + version "0.8.1" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.8.1.tgz#09e249ebde851d3b1e48d27c105444667f17b83d" + integrity sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA== + +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" + integrity sha512-zdu8XMNEDepKKR+XYOXAVPtWui0ly0NtohUscw+UmaHiAWT8hrV1rr//H6V+0DvJ3OQ19S979M0laLfX8rm82Q== + dependencies: + is-typedarray "^1.0.0" + +unicode-canonical-property-names-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz#301acdc525631670d39f6146e0e77ff6bbdebddc" + integrity sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ== + +unicode-match-property-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz#54fd16e0ecb167cf04cf1f756bdcc92eba7976c3" + integrity sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q== + dependencies: + unicode-canonical-property-names-ecmascript "^2.0.0" + unicode-property-aliases-ecmascript "^2.0.0" + +unicode-match-property-value-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.0.0.tgz#1a01aa57247c14c568b89775a54938788189a714" + integrity sha512-7Yhkc0Ye+t4PNYzOGKedDhXbYIBe1XEQYQxOPyhcXNMJ0WCABqqj6ckydd6pWRZTHV4GuCPKdBAUiMc60tsKVw== + +unicode-property-aliases-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.0.0.tgz#0a36cb9a585c4f6abd51ad1deddb285c165297c8" + integrity sha512-5Zfuy9q/DFr4tfO7ZPeVXb1aPoeQSdeFMLpYuFebehDAhbuevLs5yxSZmIFN1tP5F9Wl4IpJrYojg85/zgyZHQ== + +union-value@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/union-value/-/union-value-1.0.1.tgz#0b6fe7b835aecda61c6ea4d4f02c14221e109847" + integrity sha512-tJfXmxMeWYnczCVs7XAEvIV7ieppALdyepWMkHkwciRpZraG/xwT+s2JN8+pr1+8jCRf80FFzvr+MpQeeoF4Xg== + dependencies: + arr-union "^3.1.0" + get-value "^2.0.6" + is-extendable "^0.1.1" + set-value "^2.0.1" + +universalify@^0.1.2: + version "0.1.2" + resolved "https://registry.yarnpkg.com/universalify/-/universalify-0.1.2.tgz#b646f69be3942dabcecc9d6639c80dc105efaa66" + integrity sha512-rBJeI5CXAlmy1pV+617WB9J63U6XcazHHF2f2dbJix4XzpUF0RS3Zbj0FGIOCAva5P/d/GBOYaACQ1w+0azUkg== + +unset-value@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/unset-value/-/unset-value-1.0.0.tgz#8376873f7d2335179ffb1e6fc3a8ed0dfc8ab559" + integrity sha512-PcA2tsuGSF9cnySLHTLSh2qrQiJ70mn+r+Glzxv2TWZblxsxCC52BDlZoPCsz7STd9pN7EZetkWZBAvk4cgZdQ== + dependencies: + has-value "^0.3.1" + isobject "^3.0.0" + +urix@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/urix/-/urix-0.1.0.tgz#da937f7a62e21fec1fd18d49b35c2935067a6c72" + integrity sha512-Am1ousAhSLBeB9cG/7k7r2R0zj50uDRlZHPGbazid5s9rlF1F/QKYObEKSIunSjIOkJZqwRRLpvewjEkM7pSqg== + +use@^3.1.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/use/-/use-3.1.1.tgz#d50c8cac79a19fbc20f2911f56eb973f4e10070f" + integrity sha512-cwESVXlO3url9YWlFW/TA9cshCEhtu7IKJ/p5soJ/gGpj7vbvFrAY/eIioQ6Dw23KjZhYgiIo8HOs1nQ2vr/oQ== + +uuid@^8.3.0: + version "8.3.2" + resolved "https://registry.yarnpkg.com/uuid/-/uuid-8.3.2.tgz#80d5b5ced271bb9af6c445f21a1a04c606cefbe2" + integrity sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg== + +v8-to-istanbul@^7.0.0: + version "7.1.2" + resolved "https://registry.yarnpkg.com/v8-to-istanbul/-/v8-to-istanbul-7.1.2.tgz#30898d1a7fa0c84d225a2c1434fb958f290883c1" + integrity sha512-TxNb7YEUwkLXCQYeudi6lgQ/SZrzNO4kMdlqVxaZPUIUjCv6iSSypUQX70kNBSERpQ8fk48+d61FXk+tgqcWow== + dependencies: + "@types/istanbul-lib-coverage" "^2.0.1" + convert-source-map "^1.6.0" + source-map "^0.7.3" + +validate-npm-package-license@^3.0.1: + version "3.0.4" + resolved "https://registry.yarnpkg.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz#fc91f6b9c7ba15c857f4cb2c5defeec39d4f410a" + integrity sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew== + dependencies: + spdx-correct "^3.0.0" + spdx-expression-parse "^3.0.0" + +w3c-hr-time@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/w3c-hr-time/-/w3c-hr-time-1.0.2.tgz#0a89cdf5cc15822df9c360543676963e0cc308cd" + integrity sha512-z8P5DvDNjKDoFIHK7q8r8lackT6l+jo/Ye3HOle7l9nICP9lf1Ci25fy9vHd0JOWewkIFzXIEig3TdKT7JQ5fQ== + dependencies: + browser-process-hrtime "^1.0.0" + +w3c-xmlserializer@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/w3c-xmlserializer/-/w3c-xmlserializer-2.0.0.tgz#3e7104a05b75146cc60f564380b7f683acf1020a" + integrity sha512-4tzD0mF8iSiMiNs30BiLO3EpfGLZUT2MSX/G+o7ZywDzliWQ3OPtTZ0PTC3B3ca1UAf4cJMHB+2Bf56EriJuRA== + dependencies: + xml-name-validator "^3.0.0" + +walker@^1.0.7, walker@~1.0.5: + version "1.0.8" + resolved "https://registry.yarnpkg.com/walker/-/walker-1.0.8.tgz#bd498db477afe573dc04185f011d3ab8a8d7653f" + integrity sha512-ts/8E8l5b7kY0vlWLewOkDXMmPdLcVV4GmOQLyxuSswIJsweeFZtAsMF7k1Nszz+TYBQrlYRmzOnr398y1JemQ== + dependencies: + makeerror "1.0.12" + +webidl-conversions@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-5.0.0.tgz#ae59c8a00b121543a2acc65c0434f57b0fc11aff" + integrity sha512-VlZwKPCkYKxQgeSbH5EyngOmRp7Ww7I9rQLERETtf5ofd9pGeswWiOtogpEO850jziPRarreGxn5QIiTqpb2wA== + +webidl-conversions@^6.1.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-6.1.0.tgz#9111b4d7ea80acd40f5270d666621afa78b69514" + integrity sha512-qBIvFLGiBpLjfwmYAaHPXsn+ho5xZnGvyGvsarywGNc8VyQJUMHJ8OBKGGrPER0okBeMDaan4mNBlgBROxuI8w== + +whatwg-encoding@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/whatwg-encoding/-/whatwg-encoding-1.0.5.tgz#5abacf777c32166a51d085d6b4f3e7d27113ddb0" + integrity sha512-b5lim54JOPN9HtzvK9HFXvBma/rnfFeqsic0hSpjtDbVxR3dJKLc+KB4V6GgiGOvl7CY/KNh8rxSo9DKQrnUEw== + dependencies: + iconv-lite "0.4.24" + +whatwg-mimetype@^2.3.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/whatwg-mimetype/-/whatwg-mimetype-2.3.0.tgz#3d4b1e0312d2079879f826aff18dbeeca5960fbf" + integrity sha512-M4yMwr6mAnQz76TbJm914+gPpB/nCwvZbJU28cUD6dR004SAxDLOOSUaB1JDRqLtaOV/vi0IC5lEAGFgrjGv/g== + +whatwg-url@^8.0.0, whatwg-url@^8.5.0: + version "8.7.0" + resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-8.7.0.tgz#656a78e510ff8f3937bc0bcbe9f5c0ac35941b77" + integrity sha512-gAojqb/m9Q8a5IV96E3fHJM70AzCkgt4uXYX2O7EmuyOnLrViCQlsEBmF9UQIu3/aeAIp2U17rtbpZWNntQqdg== + dependencies: + lodash "^4.7.0" + tr46 "^2.1.0" + webidl-conversions "^6.1.0" + +which-module@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/which-module/-/which-module-2.0.0.tgz#d9ef07dce77b9902b8a3a8fa4b31c3e3f7e6e87a" + integrity sha512-B+enWhmw6cjfVC7kS8Pj9pCrKSc5txArRyaYGe088shv/FGWH+0Rjx/xPgtsWfsUtS27FkP697E4DDhgrgoc0Q== + +which@^1.2.9: + version "1.3.1" + resolved "https://registry.yarnpkg.com/which/-/which-1.3.1.tgz#a45043d54f5805316da8d62f9f50918d3da70b0a" + integrity sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ== + dependencies: + isexe "^2.0.0" + +which@^2.0.1, which@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/which/-/which-2.0.2.tgz#7c6a8dd0a636a0327e10b59c9286eee93f3f51b1" + integrity sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA== + dependencies: + isexe "^2.0.0" + +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== + +wrap-ansi@^6.2.0: + version "6.2.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-6.2.0.tgz#e9393ba07102e6c91a3b221478f0257cd2856e53" + integrity sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + +wrappy@1: + version "1.0.2" + resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" + integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== + +write-file-atomic@^3.0.0: + version "3.0.3" + resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-3.0.3.tgz#56bd5c5a5c70481cd19c571bd39ab965a5de56e8" + integrity sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q== + dependencies: + imurmurhash "^0.1.4" + is-typedarray "^1.0.0" + signal-exit "^3.0.2" + typedarray-to-buffer "^3.1.5" + +ws@^7.4.6: + version "7.5.8" + resolved "https://registry.yarnpkg.com/ws/-/ws-7.5.8.tgz#ac2729881ab9e7cbaf8787fe3469a48c5c7f636a" + integrity sha512-ri1Id1WinAX5Jqn9HejiGb8crfRio0Qgu8+MtL36rlTA6RLsMdWt1Az/19A2Qij6uSHUMphEFaTKa4WG+UNHNw== + +xml-name-validator@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/xml-name-validator/-/xml-name-validator-3.0.0.tgz#6ae73e06de4d8c6e47f9fb181f78d648ad457c6a" + integrity sha512-A5CUptxDsvxKJEU3yO6DuWBSJz/qizqzJKOMIfUJHETbBw/sFaDxgd6fxm1ewUaM0jZ444Fc5vC5ROYurg/4Pw== + +xmlchars@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/xmlchars/-/xmlchars-2.2.0.tgz#060fe1bcb7f9c76fe2a17db86a9bc3ab894210cb" + integrity sha512-JZnDKK8B0RCDw84FNdDAIpZK+JuJw+s7Lz8nksI7SIuU3UXJJslUthsi+uWBUYOwPFwW7W7PRLRfUKpxjtjFCw== + +y18n@^4.0.0: + version "4.0.3" + resolved "https://registry.yarnpkg.com/y18n/-/y18n-4.0.3.tgz#b5f259c82cd6e336921efd7bfd8bf560de9eeedf" + integrity sha512-JKhqTOwSrqNA1NY5lSztJ1GrBiUodLMmIZuLiDaMRJ+itFd+ABVE8XBjOvIWL+rSqNDC74LCSFmlb/U4UZ4hJQ== + +yallist@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" + integrity sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A== + +yargs-parser@^18.1.2: + version "18.1.3" + resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-18.1.3.tgz#be68c4975c6b2abf469236b0c870362fab09a7b0" + integrity sha512-o50j0JeToy/4K6OZcaQmW6lyXXKhq7csREXcDwk2omFPJEwUNOVtJKvmDr9EI1fAJZUyZcRF7kxGBWmRXudrCQ== + dependencies: + camelcase "^5.0.0" + decamelize "^1.2.0" + +yargs@^15.4.1: + version "15.4.1" + resolved "https://registry.yarnpkg.com/yargs/-/yargs-15.4.1.tgz#0d87a16de01aee9d8bec2bfbf74f67851730f4f8" + integrity sha512-aePbxDmcYW++PaqBsJ+HYUFwCdv4LVvdnhBy78E57PIor8/OVvhMrADFFEDh8DHDFRv/O9i3lPhsENjO7QX0+A== + dependencies: + cliui "^6.0.0" + decamelize "^1.2.0" + find-up "^4.1.0" + get-caller-file "^2.0.1" + require-directory "^2.1.1" + require-main-filename "^2.0.0" + set-blocking "^2.0.0" + string-width "^4.2.0" + which-module "^2.0.0" + y18n "^4.0.0" + yargs-parser "^18.1.2" diff --git a/website/postcss.config.js b/website/postcss.config.js new file mode 100644 index 0000000000..33ad091d26 --- /dev/null +++ b/website/postcss.config.js @@ -0,0 +1,6 @@ +module.exports = { + plugins: { + tailwindcss: {}, + autoprefixer: {}, + }, +} diff --git a/website/public/favicon-32x32.png b/website/public/favicon-32x32.png new file mode 100644 index 0000000000..0f6385096d Binary files /dev/null and b/website/public/favicon-32x32.png differ diff --git a/website/public/favicon.svg b/website/public/favicon.svg new file mode 100644 index 0000000000..c0625d0aea --- /dev/null +++ b/website/public/favicon.svg @@ -0,0 +1,161 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/public/icons/icon-144x144.png b/website/public/icons/icon-144x144.png new file mode 100644 index 0000000000..d659d9242d Binary files /dev/null and b/website/public/icons/icon-144x144.png differ diff --git a/website/public/icons/icon-192x192.png b/website/public/icons/icon-192x192.png new file mode 100644 index 0000000000..888321d651 Binary files /dev/null and b/website/public/icons/icon-192x192.png differ diff --git a/website/public/icons/icon-256x256.png b/website/public/icons/icon-256x256.png new file mode 100644 index 0000000000..a4695c0bd5 Binary files /dev/null and b/website/public/icons/icon-256x256.png differ diff --git a/website/public/icons/icon-384x384.png b/website/public/icons/icon-384x384.png new file mode 100644 index 0000000000..b508375dea Binary files /dev/null and b/website/public/icons/icon-384x384.png differ diff --git a/website/public/icons/icon-48x48.png b/website/public/icons/icon-48x48.png new file mode 100644 index 0000000000..983073e57e Binary files /dev/null and b/website/public/icons/icon-48x48.png differ diff --git a/website/public/icons/icon-512x512.png b/website/public/icons/icon-512x512.png new file mode 100644 index 0000000000..528b0cd6b8 Binary files /dev/null and b/website/public/icons/icon-512x512.png differ diff --git a/website/public/icons/icon-72x72.png b/website/public/icons/icon-72x72.png new file mode 100644 index 0000000000..8f2debd5a4 Binary files /dev/null and b/website/public/icons/icon-72x72.png differ diff --git a/website/public/icons/icon-96x96.png b/website/public/icons/icon-96x96.png new file mode 100644 index 0000000000..8f8c6b76b4 Binary files /dev/null and b/website/public/icons/icon-96x96.png differ diff --git a/website/public/manifest.webmanifest b/website/public/manifest.webmanifest new file mode 100644 index 0000000000..56c4ac835a --- /dev/null +++ b/website/public/manifest.webmanifest @@ -0,0 +1 @@ +{"name":"gatsby-starter-default","short_name":"starter","start_url":"/","background_color":"#663399","display":"minimal-ui","icons":[{"src":"icons/icon-48x48.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"48x48","type":"image/png"},{"src":"icons/icon-72x72.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"72x72","type":"image/png"},{"src":"icons/icon-96x96.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"96x96","type":"image/png"},{"src":"icons/icon-144x144.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"144x144","type":"image/png"},{"src":"icons/icon-192x192.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"192x192","type":"image/png"},{"src":"icons/icon-256x256.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"256x256","type":"image/png"},{"src":"icons/icon-384x384.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"384x384","type":"image/png"},{"src":"icons/icon-512x512.png?v=64dea6591ea7e2ffbf5b644a4b24fd1a","sizes":"512x512","type":"image/png"}]} \ No newline at end of file diff --git a/website/public/page-data/404.html/page-data.json b/website/public/page-data/404.html/page-data.json new file mode 100644 index 0000000000..26fb2cee5d --- /dev/null +++ b/website/public/page-data/404.html/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-404-js","path":"/404.html","result":{"pageContext":{}},"staticQueryHashes":["63159454"]} \ No newline at end of file diff --git a/website/public/page-data/blogs/page-data.json b/website/public/page-data/blogs/page-data.json new file mode 100644 index 0000000000..a0edb5c139 --- /dev/null +++ b/website/public/page-data/blogs/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-blogs-js","path":"/blogs/","result":{"pageContext":{}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/dev-404-page/page-data.json b/website/public/page-data/dev-404-page/page-data.json new file mode 100644 index 0000000000..7693c0abc9 --- /dev/null +++ b/website/public/page-data/dev-404-page/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---cache-dev-404-page-js","path":"/dev-404-page/","result":{"data":{"allSiteFunction":{"nodes":[]},"allSitePage":{"nodes":[{"path":"/using-dsg"},{"path":"/404/"},{"path":"/404.html"},{"path":"/blogs/"},{"path":"/download/"},{"path":"/"},{"path":"/page-2/"},{"path":"/roadmap/"},{"path":"/using-ssr/"},{"path":"/using-typescript/"},{"path":"/documents/"},{"path":"/examples/"},{"path":"/documents/5bed52d4-4216-529d-9f56-0b00e2f2f126/"},{"path":"/documents/0eef8c51-10b1-5442-8a2e-9edaed9a90e8/"},{"path":"/documents/c1b9081e-94fb-568e-9fd4-9f43308d1d1b/"},{"path":"/documents/2ea37140-1972-5012-8858-672bc7bd2751/"},{"path":"/documents/608e954e-8af3-5853-ac42-d86447b4d9a0/"},{"path":"/documents/1b353f9e-a434-50e5-bf3b-47e3ea51e808/"},{"path":"/documents/bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d/"},{"path":"/documents/f56e6cbb-9ce1-5520-b121-de0a1f687954/"},{"path":"/documents/9aff3577-2c34-5d9f-be70-b6c633a7de4d/"},{"path":"/documents/7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7/"},{"path":"/documents/b2e4dd0b-024a-5528-8281-a5e78ff075ce/"},{"path":"/documents/a948d2e4-10e5-52db-a2d3-003974ce0152/"},{"path":"/documents/db67bfa8-2060-581d-b9bd-9c1ff1c22ae8/"},{"path":"/documents/b91682e2-8005-5e87-a719-354455ac7c82/"},{"path":"/documents/71f9fc5c-a31c-5609-942b-ccd842b13e9b/"},{"path":"/documents/694ee276-4040-55d5-bf94-dd3cdbc97ddf/"},{"path":"/documents/30e430f1-c432-551a-b688-892134b2bfbc/"},{"path":"/documents/3faa50be-d6b7-500a-9b45-6163966c05a9/"},{"path":"/documents/f1ffa712-91ad-514e-b6e1-0b0eb3b064e5/"},{"path":"/documents/96216369-8952-51bc-9be6-6cf83f779728/"},{"path":"/documents/ccc9d2bb-b6a4-5a39-944b-fba076e0aaee/"},{"path":"/documents/0a1783f4-af55-5924-81c2-b48d052c3062/"},{"path":"/documents/5fb5df61-9d28-5cef-b9df-e0f47160132e/"},{"path":"/documents/9915ccf0-2301-58a5-8d78-8e30d6a1a326/"},{"path":"/documents/b3959b7e-1e67-546b-9f7f-63b4f1793bea/"},{"path":"/documents/10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178/"},{"path":"/documents/3aa8db18-e48e-5fa9-b254-1ebb5832eec2/"},{"path":"/documents/aaa33a66-9841-53ff-bcfc-b160dd7df1e2/"},{"path":"/documents/7f561cef-a143-564f-a824-7b9b92cb5d26/"},{"path":"/documents/20540b3e-20f7-5f3b-90ad-3a35718dd82f/"},{"path":"/documents/058a4b17-b273-5387-8023-9d2005d6275b/"},{"path":"/documents/cfac3f8b-1332-5242-abe4-625a31e30c03/"},{"path":"/documents/ba3c961e-85e2-5dd2-9281-2add6181b69d/"},{"path":"/documents/eada14db-1e86-5680-b7ca-2680dc53c032/"},{"path":"/documents/1f18920f-09f1-54a6-a9af-35f26fbab9c0/"},{"path":"/documents/f988eabe-e8d4-5a3f-acfd-953006152574/"},{"path":"/documents/90fcd642-5f3e-5604-968f-825b75f3b3ca/"},{"path":"/documents/4e0d7eaa-66e4-50e3-b0b3-d32c57445167/"},{"path":"/documents/5b07b05b-113b-58b1-9c50-57b41a227608/"},{"path":"/documents/b8e80fb9-b34a-5abc-a983-4490f9360f70/"},{"path":"/documents/6e373bdf-2e53-57a0-9f4a-9ae3c19edd60/"},{"path":"/documents/a4f53b25-af66-5276-aefd-b19805120a07/"},{"path":"/documents/72c7a813-5a75-5872-a575-23c527f449e7/"},{"path":"/documents/55326c3f-6bc7-5564-8de8-e8c3b42088c7/"},{"path":"/documents/7e3ba980-9692-5a2e-95f4-4f950f8de089/"},{"path":"/documents/06f14c0d-0600-5987-9274-f8087c4077b1/"},{"path":"/documents/ce4ddee7-ae18-5193-89cf-85fdfe69276a/"},{"path":"/documents/0b2dac85-2a78-542f-bfae-464b57d4bb10/"},{"path":"/documents/50096c33-6443-5aee-bc29-e3bdbcdac851/"},{"path":"/documents/ceab5595-2cad-56c9-a664-5ed386762829/"},{"path":"/documents/54bdd64a-667d-5236-9b0e-0832c34debf3/"},{"path":"/documents/31015037-3137-5747-870a-d796b9f1deb0/"},{"path":"/documents/5e4ac494-08ea-51c1-b146-42b6cff1f660/"},{"path":"/documents/4449c40d-e7c9-59b5-8e3d-d6661f0d2935/"},{"path":"/documents/88907719-b390-555d-b5f8-a0d5e481930b/"},{"path":"/documents/5c57eb91-1ef2-52fe-a0ca-a2195efa04e3/"},{"path":"/documents/2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0/"},{"path":"/documents/7cf28b4a-9eb1-50af-891a-589d659303db/"},{"path":"/documents/83e9d28d-6d36-5507-93c6-99319bd8dbca/"},{"path":"/documents/1cf60605-e0cf-5cfe-afc8-3a209fef3e70/"},{"path":"/documents/e0199b76-48f0-50c3-8da6-5d1e0c6e0529/"},{"path":"/documents/837f756f-4665-582e-8992-8a691b16ee4c/"},{"path":"/documents/7955d1c3-1727-597e-9407-1d489f6a66d8/"},{"path":"/documents/5f95d91c-7911-5590-8e0a-8d1951cc1ba1/"},{"path":"/documents/6a7a5091-fd5f-5161-a1eb-32b89a366830/"},{"path":"/documents/eef5105f-029b-54fd-97a2-f2b81a372c0f/"},{"path":"/documents/6015bcca-1157-561f-8290-3a5644d8e100/"},{"path":"/documents/e3c8934b-1838-54c5-8524-ca4bcc037c80/"},{"path":"/documents/c455aec8-9118-5f3f-953a-e909716d65f7/"},{"path":"/documents/32045ec3-89e9-5201-bff7-a73134393c7a/"},{"path":"/documents/d41a4c9e-2d11-5a2d-940c-65341ad128e7/"},{"path":"/documents/a0c640d8-5228-5fdd-bd32-437b3ca35128/"},{"path":"/documents/a2461480-7278-523a-b2c7-8b473d83bb9a/"},{"path":"/documents/960e8f11-d0a5-5985-b076-8fa7e0934d68/"},{"path":"/documents/bf2e998e-f34d-5659-a123-14fc9ffff5dc/"},{"path":"/documents/f321424a-15d0-557a-9613-943c444e0866/"},{"path":"/documents/a6da894c-8a3b-5adf-972c-ff77584d0469/"},{"path":"/documents/0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3/"},{"path":"/documents/026f8d50-34a3-58a3-9e9d-a9bff77581a2/"},{"path":"/documents/5b4c347c-ac8f-5bb0-8641-6bea56ca40bb/"},{"path":"/documents/31e148cf-6bec-57fa-a270-3b6c38dd329b/"},{"path":"/documents/d002a094-a656-56c6-974a-78823e7c2ab9/"},{"path":"/documents/be159d5f-6e5c-580e-8a41-f8c28c97ac1e/"},{"path":"/documents/02fe41db-e7e5-59bf-a02b-b8877cdce82a/"},{"path":"/documents/bfb32f91-2c2a-5668-accf-524c1301e298/"},{"path":"/documents/a64ec76c-df1f-5bed-959f-35fbf94861f8/"},{"path":"/documents/87d74717-3bc2-595e-a1e5-e05b6261aca0/"},{"path":"/documents/9fe6f140-7450-59d2-a740-ac6f20114a79/"},{"path":"/documents/56e03f9a-1b17-5bbb-9b48-28ca5e5501e2/"},{"path":"/documents/e429f8e7-c020-5f9b-9f97-1c3d5689f398/"},{"path":"/documents/c18c919c-e710-53ad-9550-af2d736a317a/"},{"path":"/documents/7f6dcc82-ea76-5e94-8f65-ddb1b1493c00/"},{"path":"/documents/485f9bc9-ef7d-5422-ba1f-5304f73e69a0/"},{"path":"/documents/e288ac0c-b263-5e39-9992-28211ec6ee16/"},{"path":"/documents/ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8/"},{"path":"/documents/1d154249-006b-528f-998e-53e7fc5158d3/"},{"path":"/documents/54a59cfe-7852-585a-a94b-0e8e0e0a4583/"},{"path":"/documents/69b00a40-e018-50d8-9274-7e3fb6eeee77/"},{"path":"/documents/ba3c4136-99cd-5297-98ad-06db24d7efcd/"},{"path":"/documents/5b02999b-9dfb-5ac3-91a1-72641f88c615/"},{"path":"/documents/418bad96-fae0-5f3d-820e-05003683e769/"},{"path":"/documents/03423183-9241-5cc9-b5c2-ffab536af8e4/"},{"path":"/documents/46f9411a-0f08-510a-8827-1220e7f587c3/"},{"path":"/documents/c4436fca-f20f-5c03-97b7-e345a3b3c1ed/"},{"path":"/documents/84d7a592-8ece-5265-96a0-1d00ab970614/"},{"path":"/documents/5c87db99-dfd3-5d33-a363-d7648a555b49/"},{"path":"/documents/ce5fc1f3-56f9-5ce8-8019-babcf30b0df2/"},{"path":"/documents/b3154ffd-5ef7-5136-a588-4aca15d33628/"},{"path":"/documents/ff49f003-a363-507c-80bd-fa62c7638d8f/"},{"path":"/documents/6522dffa-4022-5c82-bac5-546329d17960/"},{"path":"/documents/92442492-7fc5-5c30-8361-521a08d8dda6/"},{"path":"/documents/83464926-94af-5996-88a4-16a8c8577050/"},{"path":"/documents/45a3d0cc-1493-5f2d-b021-8e6e948270b6/"},{"path":"/documents/efa70c0e-a44a-516e-a527-1116abfb7609/"},{"path":"/documents/7d3b4c50-3685-566d-a7b5-e8843eb83bff/"},{"path":"/documents/91832125-df25-5124-af00-e2bf78d678ff/"},{"path":"/documents/a5ee8675-566d-5ecd-af50-911178bb7712/"},{"path":"/documents/5024a8b0-ca47-5624-9325-8ee6742fd04c/"},{"path":"/documents/e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7/"},{"path":"/documents/33492436-1819-561b-926e-ff329b246738/"},{"path":"/documents/50096837-25af-50f7-b187-11bb58fa4e87/"},{"path":"/documents/0b893e20-56cd-5ef0-931f-af5183104aa3/"},{"path":"/documents/ed763f9f-301a-5cbf-b387-5b22d4443884/"},{"path":"/documents/conector-share/"},{"path":"/documents/contribution/"},{"path":"/documents/conf-prop/"},{"path":"/documents/connector-share/"},{"path":"/documents/iceberg/"},{"path":"/documents/prometheus/"},{"path":"/documents/kerberos/"},{"path":"/documents/generalconfig/"},{"path":"/documents/quickstart/"},{"path":"/documents/questions/"},{"path":"/documents/restore/"},{"path":"/documents/statistics/"},{"path":"/documents/es5-source/"},{"path":"/documents/es5-sink/"},{"path":"/documents/es6-sink/"},{"path":"/documents/es6-lookup/"},{"path":"/documents/es6-source/"},{"path":"/documents/es7-sink/"},{"path":"/documents/es7-lookup/"},{"path":"/documents/es7-source/"},{"path":"/documents/mongodb-lookup/"},{"path":"/documents/binlog-source/"},{"path":"/documents/mongodb-source/"},{"path":"/documents/mongodb-sink/"},{"path":"/documents/postgres-cdc-source-en/"},{"path":"/documents/hdfs-sink-en/"},{"path":"/documents/hdfs-source-en/"},{"path":"/documents/mysql-lookup/"},{"path":"/documents/oracle-lookup/"},{"path":"/documents/oracle-sink/"},{"path":"/documents/oracle-source/"},{"path":"/documents/mysql-sink/"},{"path":"/documents/sqlserver-cdc-configuration/"},{"path":"/documents/mysql-source/"},{"path":"/documents/sqlserver-cdc-principles/"},{"path":"/documents/sqlserver-cdc-source/"},{"path":"/documents/sqlserver-lookup/"},{"path":"/documents/redis-lookup/"},{"path":"/documents/sqlserver-source/"},{"path":"/documents/kafka-source/"},{"path":"/documents/redis-sink/"},{"path":"/documents/kafka-sink/"},{"path":"/documents/sqlserver-sink/"},{"path":"/documents/db2-lookup/"},{"path":"/documents/cassandra-lookup/"},{"path":"/documents/cassandra-sink/"},{"path":"/documents/db2-sink/"},{"path":"/documents/db2-source/"},{"path":"/documents/dorisbatch-sink/"},{"path":"/documents/cassandra-source/"},{"path":"/documents/clickhouse-lookup/"},{"path":"/documents/clickhouse-sink/"},{"path":"/documents/clickhouse-source/"},{"path":"/documents/emqx-sink/"},{"path":"/documents/emqx-source/"},{"path":"/documents/filesystem-sink/"},{"path":"/documents/ftp-source/"},{"path":"/documents/file-source/"},{"path":"/documents/gbase-lookup/"},{"path":"/documents/ftp-sink/"},{"path":"/documents/greenplum-sink/"},{"path":"/documents/gbase-sink/"},{"path":"/documents/gbase-source/"},{"path":"/documents/greenplum-source/"},{"path":"/documents/hbase-lookup/"},{"path":"/documents/hdfs-source/"},{"path":"/documents/hdfs-sink/"},{"path":"/documents/hive-lookup/"},{"path":"/documents/hbase-sink/"},{"path":"/documents/hbase-source/"},{"path":"/documents/hive-sink/"},{"path":"/documents/http-sink/"},{"path":"/documents/http-source/"},{"path":"/documents/kingbase-sink/"},{"path":"/documents/kingbase-source/"},{"path":"/documents/log-miner-source/"},{"path":"/documents/log-miner/"},{"path":"/documents/kudu-lookup/"},{"path":"/documents/kudu-sink/"},{"path":"/documents/kudu-source/"},{"path":"/documents/postgres-cdc/"},{"path":"/documents/postgres-lookup/"},{"path":"/documents/postgres-sink/"},{"path":"/documents/postgres-source/"},{"path":"/documents/saphana-sink/"},{"path":"/documents/saphana-source/"},{"path":"/documents/solr-source/"},{"path":"/documents/solr-sink/"},{"path":"/documents/stream-sink/"},{"path":"/documents/stream-source/"},{"path":"/documents/starrocks-sink/"},{"path":"/documents/tidb-lookup/"},{"path":"/documents/tidb-sink/"},{"path":"/documents/sqlserver-cdc/"},{"path":"/examples/5e1f992d-2e11-5435-9c26-e3c8a71df04d/"}]}},"pageContext":{}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/1b353f9e-a434-50e5-bf3b-47e3ea51e808/page-data.json b/website/public/page-data/documents/1b353f9e-a434-50e5-bf3b-47e3ea51e808/page-data.json new file mode 100644 index 0000000000..b239881cca --- /dev/null +++ b/website/public/page-data/documents/1b353f9e-a434-50e5-bf3b-47e3ea51e808/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/1b353f9e-a434-50e5-bf3b-47e3ea51e808/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

使用 prometheus pushgateway 需要设置的 confProp 参数

\n
    \n
  • metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter
  • \n
  • metrics.reporter.promgateway.host: prometheus pushgateway的地址
  • \n
  • metrics.reporter.promgateway.port:prometheus pushgateway的端口
  • \n
  • metrics.reporter.promgateway.jobName: 实例名称
  • \n
  • metrics.reporter.promgateway.randomJobNameSuffix: 是否在实例名称后面添加随机字符串(默认:true)
  • \n
  • metrics.reporter.promgateway.deleteOnShutdown: 是否在停止的时候删除数据(默认false)
  • \n
","id":"7ec80874-b68b-59ac-93f2-7d203f4bf6f4","parent":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","modifiedTime":"2022-06-15T14:45:51.318Z","ino":562949955566180}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"7ec80874-b68b-59ac-93f2-7d203f4bf6f4","parent__id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","__params":{"parent__id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/5bed52d4-4216-529d-9f56-0b00e2f2f126/page-data.json b/website/public/page-data/documents/5bed52d4-4216-529d-9f56-0b00e2f2f126/page-data.json new file mode 100644 index 0000000000..09e05c381b --- /dev/null +++ b/website/public/page-data/documents/5bed52d4-4216-529d-9f56-0b00e2f2f126/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/5bed52d4-4216-529d-9f56-0b00e2f2f126/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

connector 共用

\n

NOTE:新版chunjun支持connector和flinkSql自带的connector共用。

\n

在chunjun中使用flinkSql的connector

\n
    \n
  • 本地调试\n
      \n
    • chunjun-local-test模块下将对应的flink connector的GAV拷贝到pom.xml文件中
    • \n
    • 将chunjun中的connector的GAV拷贝到pom.xml文件中(部分flink connector已经通过chunjun connector引入,如果是则上一步省略)
    • \n
    • 在LocalTest类中指定参数运行即可\r\nsql:\n
        -- {\"id\":100,\"name\":\"lb james阿道夫\",\"money\":293.899778,\"dateone\":\"2020-07-30 10:08:22\",\"age\":\"33\",\"datethree\":\"2020-07-30 10:08:22.123\",\"datesix\":\"2020-07-30 10:08:22.123456\",\"datenigth\":\"2020-07-30 10:08:22.123456789\",\"dtdate\":\"2020-07-30\",\"dttime\":\"10:08:22\"}\r\n  CREATE TABLE source_ods_fact_user_ippv \r\n  (\r\n      id INT\r\n      , name STRING\r\n      , money decimal\r\n      , dateone timestamp\r\n      , age bigint\r\n      , datethree timestamp\r\n      , datesix timestamp(6)\r\n      , datenigth timestamp(9)\r\n      , dtdate date\r\n      , dttime time\r\n  ) WITH (\r\n      'connector' = 'kafka'\r\n      ,'topic' = 'da'\r\n      ,'properties.bootstrap.servers' = 'localhost:9092'\r\n      ,'properties.group.id' = 'luna_g'\r\n      ,'scan.startup.mode' = 'earliest-offset'\r\n      ,'format' = 'json'\r\n      ,'json.timestamp-format.standard' = 'SQL'\r\n  );\r\n  \r\n  CREATE TABLE result_total_pvuv_min\r\n  (\r\n      id INT\r\n      , name STRING\r\n      , money decimal\r\n      , dateone timestamp\r\n      , age bigint\r\n      , datethree timestamp\r\n      , datesix timestamp(6)\r\n      , datenigth timestamp(9)\r\n      , dtdate date\r\n      , dttime time\r\n  ) WITH (\r\n      'connector' = 'stream-x'\r\n  );\r\n  INSERT INTO result_total_pvuv_min\r\n  SELECT *\r\n  from source_ods_fact_user_ippv;\n
      \npom:\n
      <dependency>\r\n    <groupId>org.apache.flink</groupId>\r\n    <artifactId>flink-connector-kafka_2.12</artifactId>\r\n    <version>1.12.2</version>\r\n    <exclusions>\r\n        <exclusion>\r\n        <artifactId>flink-core</artifactId>\r\n        <groupId>org.apache.flink</groupId>\r\n        </exclusion>\r\n    </exclusions>\r\n</dependency>\r\n<dependency>\r\n    <groupId>com.dtstack.chunjun</groupId>\r\n    <artifactId>chunjun-connector-stream</artifactId>\r\n    <version>1.12-SNAPSHOT</version>\r\n</dependency>\n
      \n
    • \n
    \n
  • \n
\n
\n
    \n
  • 服务器上运行:\n
      \n
    • \n

      1.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到chunjun/lib目录下,构建jobGraph使用

      \n
    • \n
    • \n

      2.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到flink/lib下

      \n
    • \n
    • \n

      3.在在chunjun/lib下目录下执行命令:

      \n
        \n
      • local模式
      • \n
      \n
      java -cp  \"./*\" com.dtstack.chunjun.client.Launcher \\\r\n-mode local \\\r\n-jobType sql \\\r\n-jobName flink1.12_SPI \\\r\n-job /yourjobpath/sqlFile.sql \\\r\n-chunjunDistDir /chunjun/chunjun-dist\n
      \n

      \"conectorShare_local.png\"

      \n
        \n
      • yarnPer模式
      • \n
      \n
      java -cp  \"./*\" com.dtstack.chunjun.client.Launcher \\\r\n-mode yarnPer \\\r\n-jobType sql \\\r\n-jobName flink1.12_SPI \\\r\n-job /yourjobpath/sqlFile.sql \\\r\n-chunjunDistDir /chunjun/chunjun-dist \\\r\n-flinkConfDir /flink/conf \\\r\n-hadoopConfDir /hadoop/etc \\\r\n-flinkLibDir /flink/lib \\\r\n-queue default\n
      \n

      \"conectorShare_yarnPer.png\"\r\n\"conectorShare_yarn.png\"

      \n
    • \n
    \n
  • \n
\n

在flinkSql中使用chunjun的connector

\n
    \n
  • 本地调试\n
      \n
    • 在自己项目中将对应的flink connector的GAV拷贝到pom.xml文件中
    • \n
    • 将chunjun中的core和connector的GAV拷贝到pom.xml文件中(需要先deploy项目)
    • \n
    • 运行自己的任务
    • \n
    \nsql:\n
        -- {\"id\":100,\"name\":\"lb james阿道夫\",\"money\":293.899778,\"dateone\":\"2020-07-30 10:08:22\",\"age\":\"33\",\"datethree\":\"2020-07-30 10:08:22.123\",\"datesix\":\"2020-07-30 10:08:22.123456\",\"datenigth\":\"2020-07-30 10:08:22.123456789\",\"dtdate\":\"2020-07-30\",\"dttime\":\"10:08:22\"}\r\n    CREATE TABLE source_ods_fact_user_ippv \r\n    (\r\n        id INT\r\n        , name STRING\r\n        , money decimal\r\n        , dateone timestamp\r\n        , age bigint\r\n        , datethree timestamp\r\n        , datesix timestamp(6)\r\n        , datenigth timestamp(9)\r\n        , dtdate date\r\n        , dttime time\r\n    ) WITH (\r\n        'connector' = 'kafka'\r\n        ,'topic' = 'da'\r\n        ,'properties.bootstrap.servers' = 'localhost:9092'\r\n        ,'properties.group.id' = 'luna_g'\r\n        ,'scan.startup.mode' = 'earliest-offset'\r\n        ,'format' = 'json'\r\n        ,'json.timestamp-format.standard' = 'SQL'\r\n    );\r\n    \r\n    CREATE TABLE result_total_pvuv_min\r\n    (\r\n        id INT\r\n        , name STRING\r\n        , money decimal\r\n        , dateone timestamp\r\n        , age bigint\r\n        , datethree timestamp\r\n        , datesix timestamp(6)\r\n        , datenigth timestamp(9)\r\n        , dtdate date\r\n        , dttime time\r\n    ) WITH (\r\n        'connector' = 'stream-x'\r\n    );\r\n    INSERT INTO result_total_pvuv_min\r\n    SELECT *\r\n    from source_ods_fact_user_ippv;\n
    \npom:\n
        <dependency>\r\n        <groupId>org.apache.flink</groupId>\r\n        <artifactId>flink-sql-connector-kafka_2.12</artifactId>\r\n        <version>1.12.2</version>\r\n    </dependency>\r\n    <dependency>\r\n        <groupId>com.dtstack.chunjun</groupId>\r\n        <artifactId>chunjun-core</artifactId>\r\n        <version>1.12-SNAPSHOT</version>\r\n    </dependency>\r\n    <dependency>\r\n        <groupId>com.dtstack.chunjun</groupId>\r\n        <artifactId>chunjun-connector-stream</artifactId>\r\n        <version>1.12-SNAPSHOT</version>\r\n    </dependency>\n
    \n
  • \n
\n
\n
    \n
  • 服务器上运行:\n
      \n
    • 将flink需要的connector和chunjun的chunjun-core-feat_1.12_pluginMerge.jar包和对应connector的jar引入到自己项目的pom中,将项目打成fat包,提交任务即可。
    • \n
    \n
  • \n
","id":"588d4c6f-b60a-550f-804f-5102fe9f5beb","parent":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","modifiedTime":"2022-06-15T14:45:51.157Z","ino":562949955565944}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"588d4c6f-b60a-550f-804f-5102fe9f5beb","parent__id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","__params":{"parent__id":"5bed52d4-4216-529d-9f56-0b00e2f2f126"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/608e954e-8af3-5853-ac42-d86447b4d9a0/page-data.json b/website/public/page-data/documents/608e954e-8af3-5853-ac42-d86447b4d9a0/page-data.json new file mode 100644 index 0000000000..2776278da9 --- /dev/null +++ b/website/public/page-data/documents/608e954e-8af3-5853-ac42-d86447b4d9a0/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/608e954e-8af3-5853-ac42-d86447b4d9a0/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

基于Iceberg master分支打包,可以在Flink 1.12下流式读写Iceberg。

\n

SQL 样例

\n

后续支持使用JSON配置ChunJun数据同步任务,将数据入湖出湖。

","id":"39930bc9-f0ff-55f3-ba68-dd06b443b6f2","parent":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","modifiedTime":"2022-06-15T14:45:51.219Z","ino":562949955566112}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"39930bc9-f0ff-55f3-ba68-dd06b443b6f2","parent__id":"608e954e-8af3-5853-ac42-d86447b4d9a0","__params":{"parent__id":"608e954e-8af3-5853-ac42-d86447b4d9a0"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7/page-data.json b/website/public/page-data/documents/7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7/page-data.json new file mode 100644 index 0000000000..8847b2b2f4 --- /dev/null +++ b/website/public/page-data/documents/7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

常见问题

\n

1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包

\n

解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装:

\n
## windows平台\r\n./install_jars.bat\r\n\r\n## unix平台\r\n./install_jars.sh\n
\n

2.ChunJun版本需要与Flink版本保持一致

\n

1.8_release版本对应flink1.8\r\n1.10_release版本对应flink1.10 版本\r\n不对应在standalone和yarn session模式提交时,会报错:\r\nCaused by: java.io.InvalidClassException: org.apache.flink.api.common.operators.ResourceSpec; incompatible types for field cpuCores

\n

3.移动ChunJun lib目录下的Launcher包后,任务启动报错:错误: 找不到或无法加载主类

\n

chunjun启动脚本里面找的是lib目录下的所有jar包,而移动后lib中含有其他的jar包,这些jar包没有主类,因此报错\r\n可以使用如下的命令运行:\r\njava -cp chunjun/lib/chunjun-client.jar com.dtstack.chunjun.launcher.Launcher -mode local -job /opt/flink/flink/deps/job/stream.json -chunjunDistDir chunjun/chunjun-dist

\n

4. 关于编译chunjun core报错Failed to read artifact descriptor for com.google.errorprone:javac-shaded

\n

报错信息:

\n
[ERROR] Failed to execute goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check (spotless-check) on project chunjun-core: \r\nExecution spotless-check of goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check failed: Unable to resolve dependencies: \r\nFailed to collect dependencies at com.google.googlejavaformat:google-java-format:jar:1.7 -> com.google.errorprone:javac-shaded:jar:9+181-r4173-1: \r\nFailed to read artifact descriptor for com.google.errorprone:javac-shaded:jar:9+181-r4173-1: Could not transfer artifact \r\ncom.google.errorprone:javac-shaded:pom:9+181-r4173-1 from/to aliyunmaven (https://maven.aliyun.com/repository/public): \r\nAccess denied to:https://maven.aliyun.com/repository/public/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.pom -> [Help 1]\n
\n

解决:

\n

https://repo1.maven.org/maven2/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.jar\r\n从这个地址下载javac-shaded-9+181-r4173-1.jar, 临时放到chunjun根目录下jars目录里,然后在源码根目录下 执行安装依赖包命令如下:

\n
mvn install:install-file -DgroupId=com.google.errorprone -DartifactId=javac-shaded -Dversion=9+181-r4173-1 -Dpackaging=jar -Dfile=./jars/javac-shaded-9+181-r4173-1.jar\n
","id":"36966c95-9eab-5abd-b1f6-d297b90bb7d2","parent":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","modifiedTime":"2022-06-15T14:45:51.319Z","ino":562949955566181}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"36966c95-9eab-5abd-b1f6-d297b90bb7d2","parent__id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","__params":{"parent__id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/binlog-source/page-data.json b/website/public/page-data/documents/binlog-source/page-data.json new file mode 100644 index 0000000000..3237d47516 --- /dev/null +++ b/website/public/page-data/documents/binlog-source/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/binlog-source/","result":{"data":{"markdownRemark":{"html":"

MySQL Binlog Source

\n\n\n\n

一、介绍

\n

MySQL Binlog插件使用Canal组件实时地从MySQL中捕获变更数据。目前sink插件暂不支持数据还原,只能写入变更的日志数据。

\n

二、支持版本

\n

MySQL 5.1.5及以上、TiDB 3.0.10之后

\n

三、插件名称

\n\n\n\n\n\n\n\n\n\n\n\n\n\n
Syncbinlogsource、binlogreader
SQLbinlog-x
\n

四、数据库配置

\n

1、修改配置文件

\n

binlog_format需要修改为 ROW 格式,在/etc/my.cnf文件里[mysqld]下添加下列配置

\n
server_id=109\r\nlog_bin = /var/lib/mysql/mysql-bin\r\nbinlog_format = ROW\r\nexpire_logs_days = 30\n
\n

2、添加权限

\n

MySQL Binlog权限需要三个权限 SELECT, REPLICATION SLAVE, REPLICATION CLIENT

\n
GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal';\n
\n
    \n
  • 缺乏SELECT权限时,报错为
  • \n
\n
com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException:\r\nAccess denied for user 'canal'@'%' to database 'binlog'\n
\n
    \n
  • 缺乏REPLICATION SLAVE权限时,报错为
  • \n
\n
java.io.IOException: \r\nError When doing Register slave:ErrorPacket [errorNumber=1045, fieldCount=-1, message=Access denied for user 'canal'@'%'\n
\n
    \n
  • 缺乏REPLICATION CLIENT权限时,报错为
  • \n
\n
com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: \r\nAccess denied; you need (at least one of) the SUPER, REPLICATION CLIENT privilege(s) for this operation\n
\n

Binlog为什么需要这些权限:

\n
    \n
  • Select权限代表允许从表中查看数据
  • \n
  • Replication client权限代表允许执行show master status,show slave status,show binary logs命令
  • \n
  • Replication slave权限代表允许slave主机通过此用户连接master以便建立主从 复制关系
  • \n
\n

五、参数说明

\n

1、Sync

\n
    \n
  • \n

    jdbcUrl

    \n
      \n
    • 描述:MySQL数据库的jdbc连接字符串,参考文档:Mysql官方文档
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    username

    \n
      \n
    • 描述:数据源的用户名
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    password

    \n
      \n
    • 描述:数据源指定用户名的密码
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    host

    \n
      \n
    • 描述:启动MySQL slave的机器ip
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    port

    \n
      \n
    • 描述:启动MySQL slave的端口
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:3306\r\n
    • \n
    \n
  • \n
  • \n

    table

    \n
      \n
    • 描述:需要解析的数据表。
    • \n
    • 注意:指定此参数后filter参数将无效,table和filter都为空,监听jdbcUrl里的schema下所有表
    • \n
    • 必选:否
    • \n
    • 字段类型:list
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    filter

    \n
      \n
    • 描述:过滤表名的Perl正则表达式
    • \n
    • 注意:table和filter都为空,监听jdbcUrl里的schema下所有表
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无
    • \n
    • 例子:\n
        \n
      • 所有表:.* or .*\\\\..*
      • \n
      • canal schema下所有表: canal\\\\..*
      • \n
      • canal下的以canal打头的表:canal\\\\.canal.*
      • \n
      • canal schema下的一张表:canal.test1\r\n
      • \n
      \n
    • \n
    \n
  • \n
  • \n

    cat

    \n
      \n
    • 描述:需要解析的数据更新类型,包括insert、update、delete三种
    • \n
    • 注意:以英文逗号分割的格式填写。如果为空,解析所有数据更新类型
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    start

    \n
      \n
    • 描述:要读取的binlog文件的开始位置
    • \n
    • 注意:为空,则从当前position处消费,timestamp的优先级高于 journalName+position
    • \n
    • 参数:\n
        \n
      • timestamp:时间戳,采集起点从指定的时间戳处消费;
      • \n
      • journalName:文件名,采集起点从指定文件的起始处消费;
      • \n
      • position:文件的指定位置,采集起点从指定文件的指定位置处消费
      • \n
      \n
    • \n
    • 字段类型:map
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    transactionSize

    \n
      \n
    • 描述:DML数据缓存大小。
    • \n
    • 注意:
    • \n
    • 必选:否
    • \n
    • 默认值:1024
    • \n
    \n
  • \n
  • \n

    pavingData

    \n
      \n
    • 描述:是否将解析出的json数据拍平,具体见六、数据结构
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    splitUpdate

    \n
      \n
    • 描述:当数据更新类型为update时,是否将update拆分为两条数据,具体见六、数据结构
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:false\r\n
    • \n
    \n
  • \n
  • \n

    timestampFormat

    \n
      \n
    • 描述:指定输入输出所使用的timestamp格式,可选值:SQLISO_8601
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:SQL\r\n
    • \n
    \n
  • \n
  • \n

    slaveId

    \n
      \n
    • 描述:从服务器的ID
    • \n
    • 注意:同一个MYSQL复制组内不能重复
    • \n
    • 必选:否
    • \n
    • 字段类型:long
    • \n
    • 默认值:new Object().hashCode()\r\n
    • \n
    \n
  • \n
  • \n

    connectionCharset

    \n
      \n
    • 描述:编码信息
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:UTF-8\r\n
    • \n
    \n
  • \n
  • \n

    detectingEnable

    \n
      \n
    • 描述:是否开启心跳
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    detectingSQL

    \n
      \n
    • 描述:心跳SQL
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:SELECT CURRENT_DATE\r\n
    • \n
    \n
  • \n
  • \n

    enableTsdb

    \n
      \n
    • 描述:是否开启时序表结构能力
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    bufferSize

    \n
      \n
    • 描述:并发缓存大小
    • \n
    • 注意:必须为2的幂
    • \n
    • 必选:否
    • \n
    • 默认值:1024\r\n
    • \n
    \n
  • \n
  • \n

    parallel

    \n
      \n
    • 描述:是否开启并行解析binlog日志
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    parallelThreadSize

    \n
      \n
    • 描述:并行解析binlog日志线程数
    • \n
    • 注意:只有 paraller 设置为true才生效
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:2\r\n
    • \n
    \n
  • \n
  • \n

    isGTIDMode

    \n
      \n
    • 描述:是否开启gtid模式
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:false\r\n
    • \n
    \n
  • \n
  • \n

    queryTimeOut

    \n
      \n
    • 描述:通过TCP连接发送数据(在这里就是要执行的sql)后,等待响应的超时时间,单位毫秒
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:300000\r\n
    • \n
    \n
  • \n
  • \n

    connectTimeOut

    \n
      \n
    • 描述:数据库驱动(mysql-connector-java)与mysql服务器建立TCP连接的超时时间,单位毫秒
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:60000\r\n
    • \n
    \n
  • \n
\n

2、SQL

\n
    \n
  • \n

    url

    \n
      \n
    • 描述:MySQL数据库的jdbc连接字符串,参考文档:Mysql官方文档
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    username

    \n
      \n
    • 描述:数据源的用户名
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    password

    \n
      \n
    • 描述:数据源指定用户名的密码
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    host

    \n
      \n
    • 描述:启动MySQL slave的机器ip
    • \n
    • 必选:是
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    port

    \n
      \n
    • 描述:启动MySQL slave的端口
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:3306\r\n
    • \n
    \n
  • \n
  • \n

    table

    \n
      \n
    • 描述:需要解析的数据表。
    • \n
    • 注意:指定此参数后filter参数将无效,SQL任务只支持监听单张表
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    filter

    \n
      \n
    • 描述:过滤表名的Perl正则表达式
    • \n
    • 注意:SQL任务只支持监听单张表
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无
    • \n
    • 例子:canal schema下的一张表:canal.test1\r\n
    • \n
    \n
  • \n
  • \n

    cat

    \n
      \n
    • 描述:需要解析的数据更新类型,包括insert、update、delete三种
    • \n
    • 注意:以英文逗号分割的格式填写。如果为空,解析所有数据更新类型
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    timestamp

    \n
      \n
    • 描述:要读取的binlog文件的开始位置,时间戳,采集起点从指定的时间戳处消费;
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    journal-name

    \n
      \n
    • 描述:要读取的binlog文件的开始位置,文件名,采集起点从指定文件的起始处消费;
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    position

    \n
      \n
    • 描述:要读取的binlog文件的开始位置,文件的指定位置,采集起点从指定文件的指定位置处消费
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:无\r\n
    • \n
    \n
  • \n
  • \n

    connection-charset

    \n
      \n
    • 描述:编码信息
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:UTF-8\r\n
    • \n
    \n
  • \n
  • \n

    detecting-enable

    \n
      \n
    • 描述:是否开启心跳
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    detecting-sql

    \n
      \n
    • 描述:心跳SQL
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:SELECT CURRENT_DATE\r\n
    • \n
    \n
  • \n
  • \n

    enable-tsdb

    \n
      \n
    • 描述:是否开启时序表结构能力
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    buffer-size

    \n
      \n
    • 描述:并发缓存大小
    • \n
    • 注意:必须为2的幂
    • \n
    • 必选:否
    • \n
    • 默认值:1024\r\n
    • \n
    \n
  • \n
  • \n

    parallel

    \n
      \n
    • 描述:是否开启并行解析binlog日志
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:true\r\n
    • \n
    \n
  • \n
  • \n

    parallel-thread-size

    \n
      \n
    • 描述:并行解析binlog日志线程数
    • \n
    • 注意:只有 paraller 设置为true才生效
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:2\r\n
    • \n
    \n
  • \n
  • \n

    is-gtid-mode

    \n
      \n
    • 描述:是否开启gtid模式
    • \n
    • 必选:否
    • \n
    • 字段类型:boolean
    • \n
    • 默认值:false\r\n
    • \n
    \n
  • \n
  • \n

    query-time-out

    \n
      \n
    • 描述:通过TCP连接发送数据(在这里就是要执行的sql)后,等待响应的超时时间,单位毫秒
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:300000\r\n
    • \n
    \n
  • \n
  • \n

    connect-time-out

    \n
      \n
    • 描述:数据库驱动(mysql-connector-java)与mysql服务器建立TCP连接的超时时间,单位毫秒
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:60000\r\n
    • \n
    \n
  • \n
  • \n

    timestamp-format.standard

    \n
      \n
    • 描述:同Sync中的timestampFormat参数,指定输入输出所使用的timestamp格式,可选值:SQLISO_8601
    • \n
    • 必选:否
    • \n
    • 字段类型:string
    • \n
    • 默认值:SQL
    • \n
    \n
  • \n
\n

六、数据结构

\n

在2020-01-01 12:30:00(时间戳:1577853000000)执行:

\n
INSERT INTO `tudou`.`kudu`(`id`, `user_id`, `name`) VALUES (1, 1, 'a');\n
\n

在2020-01-01 12:31:00(时间戳:1577853060000)执行:

\n
DELETE FROM `tudou`.`kudu` WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a';\n
\n

在2020-01-01 12:32:00(时间戳:1577853180000)执行:

\n
UPDATE `tudou`.`kudu` SET `id` = 2, `user_id` = 2, `name` = 'b' WHERE `id` = 1 AND `user_id` = 1 AND `name` = 'a';\n
\n

1、pavingData = true, splitUpdate = false\r\nRowData中的数据依次为:

\n
//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853000000, \"INSERT\", null, null, null, 1, 1, \"a\"]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853060000, \"DELETE\", 1, 1, \"a\", null, null, null]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE\", 1, 1, \"a\", 2, 2, \"b\"]\n
\n

2、pavingData = false, splitUpdate = false\r\nRowData中的数据依次为:

\n
//schema, table, ts, opTime, type, before, after\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853000000, \"INSERT\", null, {\"id\":1, \"user_id\":1, \"name\":\"a\"}]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853060000, \"DELETE\", {\"id\":1, \"user_id\":1, \"name\":\"a\"}, null]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE\", {\"id\":1, \"user_id\":1, \"name\":\"a\"}, {\"id\":2, \"user_id\":2, \"name\":\"b\"}]\n
\n

3、pavingData = true, splitUpdate = true\r\nRowData中的数据依次为:

\n
//schema, table, ts, opTime, type, before_id, before_user_id, before_name, after_id, after_user_id, after_name\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853000000, \"INSERT\", null, null, null, 1, 1, \"a\"]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853060000, \"DELETE\", 1, 1, \"a\", null, null, null]\r\n\r\n//schema, table, ts, opTime, type, before_id, before_user_id, before_name\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE_BEFORE\", 1, 1, \"a\"]\r\n\r\n//schema, table, ts, opTime, type, after_id, after_user_id, after_name\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE_AFTER\", 2, 2, \"b\"]\n
\n

4、pavingData = false, splitUpdate = true\r\nRowData中的数据依次为:

\n
//schema, table, ts, opTime, type, before, after\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853000000, \"INSERT\", null, {\"id\":1, \"user_id\":1, \"name\":\"a\"}]\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853060000, \"DELETE\", {\"id\":1, \"user_id\":1, \"name\":\"a\"}, null]\r\n//schema, table, ts, opTime, type, before\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE_BEFORE\", {\"id\":1, \"user_id\":1, \"name\":\"a\"}]\r\n//schema, table, ts, opTime, type, after\r\n[\"tudou\", \"kudu\", 6760525407742726144, 1577853180000, \"UPDATE_AFTER\", {\"id\":2, \"user_id\":2, \"name\":\"b\"}]\n
\n
    \n
  • type:变更类型,INSERT,UPDATE、DELETE
  • \n
  • opTime:数据库中SQL的执行时间
  • \n
  • ts:自增ID,不重复,可用于排序,解码后为ChunJun的事件时间,解码规则如下:
  • \n
\n
long id = Long.parseLong(\"6760525407742726144\");\r\nlong res = id >> 22;\r\nDateFormat sdf = new SimpleDateFormat(\"yyyy-MM-dd HH:mm:ss\");\r\nSystem.out.println(sdf.format(res));\t\t//2021-01-28 19:54:21\n
\n

七、数据类型

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
支持BIT
TINYINT、SMALLINT、MEDIUMINT、INT、INT24、INTEGER、FLOAT、DOUBLE、REAL、LONG、BIGINT、DECIMAL、NUMERIC
CHAR、VARCHAR、TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT、ENUM、SET、JSON
DATE、TIME、TIMESTAMP、DATETIME、YEAR
TINYBLOB、BLOB、MEDIUMBLOB、LONGBLOB、GEOMETRY、BINARY、VARBINARY
暂不支持
\n

八、脚本示例

\n

见项目内chunjun-examples文件夹。

","id":"8fb3d3a7-1017-5951-8a21-906dea8d123a","parent":{"id":"ec0b3e58-5cea-5bee-a0c5-77e6369a3905","name":"binlog-source","modifiedTime":"2022-06-07T13:14:11.895Z","ino":3377699721738917}}},"pageContext":{"id":"8fb3d3a7-1017-5951-8a21-906dea8d123a","parent__name":"binlog-source","__params":{"parent__name":"binlog-source"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/c1b9081e-94fb-568e-9fd4-9f43308d1d1b/page-data.json b/website/public/page-data/documents/c1b9081e-94fb-568e-9fd4-9f43308d1d1b/page-data.json new file mode 100644 index 0000000000..2e4eb22416 --- /dev/null +++ b/website/public/page-data/documents/c1b9081e-94fb-568e-9fd4-9f43308d1d1b/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/c1b9081e-94fb-568e-9fd4-9f43308d1d1b/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

confProp

\n
    \n
  • \n

    table.exec.source.idle-timeout:当一个源在超时时间内没有收到任何元素时,它将被标记为临时空闲。这允许下游任务推进其水印,而无需在空闲时等待来自该源的水印。默认值为 0(表示未启用检测源空闲),可设置:10 ms(单位毫秒)。

    \n
  • \n
  • \n

    table.exec.emit.early-fire.enabled:开启window统计提前触发功能。默认:false(表示不开启),设置true开启。

    \n
  • \n
  • \n

    table.exec.emit.early-fire.delay:开启window统计提前触发时间,上面设置为true才有效。无默认值,可设置:1s(单位为秒)。

    \n
  • \n
  • \n

    table.exec.state.ttl:状态最小过期时间。默认:0 ms(代表不过期)。

    \n
  • \n
  • \n

    table.exec.mini-batch.enabled:是否开启minibatch,可以减少状态开销。这可能会增加一些延迟,因为它会缓冲一些记录而不是立即处理它们。这是吞吐量和延迟之间的权衡。默认:false(表示不开启),设置true开启

    \n
  • \n
  • \n

    table.exec.mini-batch.allow-latency:状态缓存时间,table.exec.mini-batch.enabled为true才有效。无默认,可设置:5 s(单位为秒)。

    \n
  • \n
  • \n

    table.exec.mini-batch.size:状态最大缓存条数,table.exec.mini-batch.enabled为true才有效。无默认,可设置:5000(单位为条数)。

    \n
  • \n
  • \n

    table.optimizer.agg-phase-strategy:是否开启Local-Global聚合,前提需要开启minibatch,聚合是为解决数据倾斜问题提出的,类似于 MapReduce 中的 Combine + Reduce 模式。无默认,可设置:TWO_PHASE。

    \n
  • \n
  • \n

    table.optimizer.distinct-agg.split.enabled:是否开启拆分distinct聚合,Local-Global可以解决数据倾斜,但是在处理distinct聚合时,其性能并不令人满意,如:SELECT day, COUNT(DISTINCT user_id) FROM T GROUP BY day 如果 distinct key (即 user_id)的值分布稀疏,建议开启。无默认,可设置:true。

    \n
  • \n
  • \n

    其他一些sql相关配置参考:https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/table/config.html

    \n
  • \n
  • \n

    sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms)

    \n
  • \n
  • \n

    sql.checkpoint.unalignedCheckpoints:是否开启Unaligned Checkpoint,不开启false,开启true。默认为:false。

    \n
  • \n
  • \n

    sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE]

    \n
  • \n
  • \n

    sql.checkpoint.timeout: 生成checkpoint的超时时间(ms)

    \n
  • \n
  • \n

    sql.max.concurrent.checkpoints: 最大并发生成checkpoint数

    \n
  • \n
  • \n

    sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)]

    \n
  • \n
  • \n

    state.backend: 任务状态后端,可选为MEMORY,FILESYSTEM,ROCKSDB,默认为flinkconf中的配置。

    \n
  • \n
  • \n

    state.checkpoints.dir: FILESYSTEM,ROCKSDB状态后端文件系统存储路径,例如:hdfs://ns1/dtInsight/flink180/checkpoints。

    \n
  • \n
  • \n

    state.backend.incremental: ROCKSDB状态后端是否开启增量checkpoint,默认为true。

    \n
  • \n
  • \n

    其他一些state相关配置参考:https://ci.apache.org/projects/flink/flink-docs-release-1.12/zh/dev/stream/state/checkpointing.html

    \n
  • \n
  • \n

    sql.env.parallelism: 默认并行度设置

    \n
  • \n
  • \n

    sql.max.env.parallelism: 最大并行度设置

    \n
  • \n
  • \n

    time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime]

    \n
  • \n
  • \n

    jobmanager.memory.process.size: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:1600m)

    \n
  • \n
  • \n

    taskmanager.memory.process.size: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:1728m)

    \n
  • \n
  • \n

    taskmanager.numberOfTaskSlots:per_job模式下指定每个taskmanager对应的slot数量(默认1),通过该参数和sql.env.parallelism可控制tm的个数,即sql.env.parallelism/taskmanager.numberOfTaskSlots 向上取整。

    \n
  • \n
  • \n

    s:任务恢复点的路径(默认无)

    \n
  • \n
  • \n

    allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false)

    \n
  • \n
  • \n

    logLevel: 日志级别动态配置(默认info)

    \n
  • \n
  • \n

    prometheus 相关参数 per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例

    \n
  • \n
","id":"f2029a3b-e951-5f7b-b2c0-fb10eb744c76","parent":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","modifiedTime":"2022-06-15T14:45:51.158Z","ino":562949955565945}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"f2029a3b-e951-5f7b-b2c0-fb10eb744c76","parent__id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","__params":{"parent__id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/clickhouse-sink/page-data.json b/website/public/page-data/documents/clickhouse-sink/page-data.json new file mode 100644 index 0000000000..7b8196f19a --- /dev/null +++ b/website/public/page-data/documents/clickhouse-sink/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/clickhouse-sink/","result":{"data":{"markdownRemark":{"html":"

Clickhouse Sink

\n

一、介绍

\n

clickhouse sink

\n

二、支持版本

\n

ClickHouse 19.x及以上

\n

三、插件名称

\n\n\n\n\n\n\n\n\n\n\n\n\n\n
SYNCclickhousesink, clickhousewriter
SQLclickhouse-x
\n

四、参数说明

\n

1、sync

\n
    \n
  • connection\n
      \n
    • 描述:数据库连接参数,包含jdbcUrl、schema、table参数
    • \n
    • 必选:是
    • \n
    • 字段类型:List
    • \n
    • 默认值:无\n
        \n
      • 示例:指定jdbcUrl、schema、table
      • \n
      \n
    • \n
    \n
  • \n
\n
\"connection\": [{\r\n     \"jdbcUrl\": [\"jdbc:clickhouse://localhost:8123/default\"],\r\n  \t \"schema\": \"public\",\r\n     \"table\": [\"table\"]\r\n    }]\n
\n

​\r\n

\n
    \n
  • jdbcUrl\n\n
  • \n
\n
\n
    \n
  • schema\n
      \n
    • 描述:数据库schema名
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n

​\r\n

\n
    \n
  • table\n
      \n
    • 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表
    • \n
    • 必选:是
    • \n
    • 字段类型:List
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • username\n
      \n
    • 描述:数据源的用户名
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • password\n
      \n
    • 描述:数据源指定用户名的密码
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • column\n
      \n
    • 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: \"column\": [\"id\",\"name\",\"age\"]
    • \n
    • 必选:是
    • \n
    • 字段类型:List
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • fullcolumn\n
      \n
    • 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: \"column\": [\"id\",\"name\",\"age\",\"hobby\"],如果不配置,将在系统表中获取
    • \n
    • 必选:否
    • \n
    • 字段类型:List
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • preSql\n
      \n
    • 描述:写入数据到目的表前,会先执行这里的一组标准语句
    • \n
    • 必选:否
    • \n
    • 字段类型:List
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • postSql\n
      \n
    • 描述:写入数据到目的表后,会执行这里的一组标准语句
    • \n
    • 必选:否
    • \n
    • 字段类型:List
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • writeMode\n
      \n
    • 描述:控制写入数据到目标表采用 insert into 或者 replace into 或者 ON DUPLICATE KEY UPDATE 语句
    • \n
    • 必选:否
    • \n
    • 所有选项:insert
    • \n
    • 字段类型:String
    • \n
    • 默认值:insert
    • \n
    \n
  • \n
\n


\n
    \n
  • batchSize\n
      \n
    • 描述:一次性批量提交的记录数大小,该值可以极大减少ChunJun与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成ChunJun运行进程OOM情况
    • \n
    • 必选:否
    • \n
    • 字段类型:int
    • \n
    • 默认值:1024
    • \n
    \n
  • \n
\n


\n
    \n
  • semantic\n
      \n
    • 描述:sink端是否支持二阶段提交
    • \n
    • 注意:\n
        \n
      • 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义;
      • \n
      • 当前只支持exactly-once 和at-least-once
      • \n
      \n
    • \n
    • 必选:否
    • \n
    • 参数类型:String\n
        \n
      • 示例:\"semantic\": \"exactly-once\"
      • \n
      \n
    • \n
    • 默认值:at-least-once
    • \n
    \n
  • \n
\n
\n

2、sql

\n
    \n
  • connector\n
      \n
    • 描述:clickhouse-x
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • url\n
      \n
    • 描述:clickhouse jdbc url
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • table-name\n
      \n
    • 描述:表名
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • username\n
      \n
    • 描述:用户名
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • password\n
      \n
    • 描述:密码
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • password\n
      \n
    • 描述:密码
    • \n
    • 必选:是
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n


\n
    \n
  • sink.buffer-flush.max-rows\n
      \n
    • 描述:批量写数据条数,单位:条
    • \n
    • 必选:否
    • \n
    • 字段类型:String
    • \n
    • 默认值:1024
    • \n
    \n
  • \n
\n


\n
    \n
  • sink.buffer-flush.interval\n
      \n
    • 描述:批量写时间间隔,单位:毫秒
    • \n
    • 必选:否
    • \n
    • 字段类型:String
    • \n
    • 默认值:10000
    • \n
    \n
  • \n
\n


\n
    \n
  • sink.all-replace\n
      \n
    • 描述:是否全部替换数据库中的数据(如果数据库中原值不为null,新值为null,如果为true则会替换为null)
    • \n
    • 必选:否
    • \n
    • 字段类型:String
    • \n
    • 默认值:false
    • \n
    \n
  • \n
\n


\n
    \n
  • sink.semantic\n
      \n
    • 描述:sink端是否支持二阶段提交
    • \n
    • 注意:\n
        \n
      • 如果此参数为空,默认不开启二阶段提交,即sink端不支持exactly_once语义;
      • \n
      • 当前只支持exactly-once 和at-least-once
      • \n
      \n
    • \n
    • 必选:否
    • \n
    • 参数类型:String\n
        \n
      • 示例:\"semantic\": \"exactly-once\"
      • \n
      \n
    • \n
    • 默认值:at-least-once
    • \n
    \n
  • \n
\n
\n
    \n
  • sink.parallelism\n
      \n
    • 描述:写入结果的并行度
    • \n
    • 必选:否
    • \n
    • 字段类型:String
    • \n
    • 默认值:无
    • \n
    \n
  • \n
\n

五、数据类型

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
支持BOOLEAN
TINYINT
SMALLINT
INT
BIGINT
FLOAT
DOUBLE
DECIMAL
STRING
VARCHAR
CHAR
TIMESTAMP
DATE
BINARY
NULL
暂不支持ARRAY
MAP
STRUCT
UNION
\n

六、配置示例

\n

见项目内chunjun-examples文件夹。

","id":"2b016cfc-ffb8-5869-9035-617d77eea70e","parent":{"id":"05e6b3ee-86c1-55d7-b373-ff04bfd45262","name":"clickhouse-sink","modifiedTime":"2022-06-07T13:14:11.895Z","ino":3377699721738924}}},"pageContext":{"id":"2b016cfc-ffb8-5869-9035-617d77eea70e","parent__name":"clickhouse-sink","__params":{"parent__name":"clickhouse-sink"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/contribution/page-data.json b/website/public/page-data/documents/contribution/page-data.json new file mode 100644 index 0000000000..b7eea7c24c --- /dev/null +++ b/website/public/page-data/documents/contribution/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/contribution/","result":{"data":{"markdownRemark":{"html":"

如何贡献ChunJun

\n

本文面向ChunJun插件开发人员,尝试通过一个开发者的角度尽可能全面地阐述一个ChunJun插件所经历的过程,同时消除开发者的困惑,快速上手插件开发。

\n

从数据流的角度来看ChunJun,可以理解为不同数据源的数据流通过对应的ChunJun插件处理,变成符合ChunJun数据规范的数据流;脏数据的处理可以理解为脏水流通过污水处理厂,变成符合标准,可以使用的水流,而对不能处理的水流收集起来。

\n

插件开发不需要关注任务具体如何调度,只需要关注关键问题:

\n
    \n
  1. 数据源本身读写数据的正确性;
  2. \n
  3. 如何合理且正确地使用框架;
  4. \n
  5. 配置文件的规范;
  6. \n
\n

\n

开发环境

\n
    \n
  • Flink集群: 1.4及以上(单机模式不需要安装Flink集群)
  • \n
  • Java: JDK8及以上
  • \n
  • 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。
  • \n
\n

开发之前,需要理解以下概念:

\n

\n

逻辑执行概念

\n

插件开发者不需要关心太多整个框架的具体运行,只需要关注数据源的读写,以及代码在逻辑上是怎么被执行的,方法什么时候被调用的。以下概念的理解对你快速开发会有帮助:

\n
    \n
  • Job:** Job**是ChunJun用以描述从一个源头到一个目的端的同步作业,是ChunJun数据同步的最小业务单元。
  • \n
  • Internal: 把Job拆分得到的最小执行单元。
  • \n
  • InputSplit:数据切片,是进入Internal的最小数据流单位。里面包含了基本数据信息和统计信息。
  • \n
  • InputFormat:读插件的执行单位。
  • \n
  • OutputFormat:写插件的执行单位。
  • \n
\n

\n

任务执行模式

\n
    \n
  • 单机模式:对应Flink集群的单机模式
  • \n
  • standalone模式:对应Flink集群的分布式模式
  • \n
  • yarn模式:对应Flink集群的yarn模式
  • \n
  • yarnPer模式: 对应Flink集群的Per-job模式
  • \n
\n

在实际开发中,上述几种模式对插件的编写没有过多的影响,一般在本地LocalTest通过,将任务上传到Flink集群测试没有什么大问题。

\n

\n

插件入口类

\n

插件的入口类需继承DataReaderDataWriter,在内部获取任务json传来的参数,通过相应的Builder构建对应InputFormatOutputFormat实例

\n

\n

DataReader

\n
public class SomeReader extends DataReader {\r\n    protected String oneParameter;\r\n    public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) {\r\n        super(config, env);\r\n    }\r\n    @Override\r\n    public DataStream<Record> readData() {\r\n        return null;\r\n    }\r\n}\n
\n

reader类需继承DataReader,同时重写readData方法。在构造函数中获取任务json中构建InputFormat所需要的参数,代码案例如下:

\n

构造方法

\n
protected String oneParameter;\r\npublic SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) {\r\n    super(config, env);\r\n    // 首先通过jobconfig获取任务json中reader信息\r\n    ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader();\r\n    // 通过getParameter()获取相应的参数信息\r\n    oneParameter = readerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER);\r\n}\n
\n

重写readData方法

\n
@Override\r\npublic DataStream<Record> readData() {\r\n    // 通过Builder构建InputFormat\r\n    SomeInputFormatBuilder builder = new SomeInputFormatBuilder(new SomeInputFormat());\r\n    // 一个setOneParameter()方法只set一个参数\r\n    builder.setOneParameter(OneParameter);\r\n    //调用createInput返回一个DataStream实例\r\n    return createInput(builder.finish());\r\n}\n
\n

\n

DataWriter

\n
public class SomeWriter extends DataWriter {\r\n    protected String oneParameter;\r\n    public SomeWriter(DataTransferConfig config) {\r\n        super(config);\r\n    }\r\n\r\n    @Override\r\n    public DataStreamSink<?> writeData(DataStream<Record> dataSet) {\r\n        return null;\r\n    }\r\n}\n
\n

和DataReader类似,writer需继承DataWriter,同时重写writeData方法。通常会创建一个ConfigKeys类,包含reader和writer所有需要的使用的任务json中参数的key。

\n

构造方法

\n
protected String oneParameter;\r\npublic SomeWriter(DataTransferConfig config) {\r\n    super(config);\r\n    // 首先通过jobconfig获取jobjson中writer信息\r\n    WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter();\r\n\toneParameter = writerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER);\r\n}\n
\n

重写writeData方法

\n
@Override\r\npublic DataStreamSink<?> writeData(DataStream<Record> dataSet) {\r\n    // 通过Builder构建OutputFormat\r\n    SomeOutputFormatBuilder builder = new SomeOutputFormatBuilder(new SomeOutputFormat());\r\n    // 一个setOneParameter()方法只set一个参数\r\n    builder.setOneParameter(OneParameter);\r\n    //调用createInput返回一个DataSink实例\r\n    return createInput(builder.finish());\r\n}\n
\n

\n

InputFormatBuilder的设计

\n

需继承RichInputFormatBuilder

\n
public class SomeInputFormatBuilder extends RichInputFormatBuilder {\r\n    /**\r\n    * 首先实例化一个InputFormat实例,通过构造函数传递,通过set方法设置参数\r\n    */\r\n    protected SomeInputFormat format;\r\n    //InputFormat构造函数,需要给实例化父类的format\r\n    public SomeInputFormatBuilder(SomeInputFormat format){\r\n        super.format = this.format = format;\r\n    }\r\n    //set方法示例,建议set方法内只给一个变量赋值\r\n    public void setOneParameter(String oneParameter){\r\n        this.oneParameter = oneParameter;\r\n    }\r\n    //重写checkFormat,检查一些必要参数设置是否正确\r\n    @Override\r\n    protected void checkFormat() {}\r\n}\n
\n

\n

InputFormat的设计

\n

需继承RichInputFormat,根据任务逻辑分别实现

\n
public class SomeInputFormat extends RichInputFormat {\r\n    @override\r\n    public void openInputFormat() {\r\n        \r\n    }\r\n    \r\n    @override\r\n    public void closeInputFormat() {\r\n    }\r\n    \r\n    @Override\r\n    protected void openInternal(InputSplit inputSplit) throws IOException {\r\n    }\r\n\r\n    @Override\r\n    protected Row nextRecordInternal(Row row) throws IOException {\r\n        return null;\r\n    }\r\n\r\n    @Override\r\n    protected void closeInternal() throws IOException {\r\n    }\r\n\r\n    @Override\r\n    public void configure(Configuration parameters) {\r\n    }\r\n\r\n    @Override\r\n    public InputSplit[] createInputSplits(int minNumSplits) throws IOException {\r\n        return new InputSplit[0];\r\n    }\r\n\r\n    @Override\r\n    public boolean reachedEnd() throws IOException {\r\n        return false;\r\n    }\r\n}\n
\n

方法功能如下:\r\n

\n

configure

\n
    \n
  • 调用位置:configure方法会在JobManager里构建执行计划的时候和在TaskManager里初始化并发实例后各调用一次;
  • \n
  • 作用:用于配置task的实例;
  • \n
  • 注意事项:不要在这个方法里写耗时的逻辑,比如获取连接,运行sql等,否则可能会导致akka超
  • \n
\n

\n

createInputSplits

\n
    \n
  • 调用位置:在构建执行计划时调用;
  • \n
  • 作用:调用子类的逻辑生成数据分片;
  • \n
  • 注意事项:分片的数量和并发数没有严格对应关系,不要在这个方法里做耗时的操作,否则会导致akka超时异常;
  • \n
\n

\n

getInputSplitAssigner

\n
    \n
  • 调用位置:创建分片后调用;
  • \n
  • 作用:获取分片分配器,同步插件里使用的是DefaultInputSplitAssigner,按顺序返回分配给各个并发实例;
  • \n
  • 注意事项:无;
  • \n
\n

\n

openInternal

\n
    \n
  • 调用位置:开始读取分片时调用;
  • \n
  • 作用:用于打开需要读取的数据源,并做一些初始化;
  • \n
  • 注意事项:这个方法必须是可以重复调用的,因为同一个并发实例可能会处理多个分片;
  • \n
\n

\n

reachEnd和nextRecordInternal

\n
    \n
  • 调用位置:任务运行时,读取每条数据时调用;
  • \n
  • 作用:返回结束标识和下一条记录;
  • \n
  • 注意事项:无
  • \n
\n

\n

closeInternal

\n
    \n
  • 调用位置:读取完一个分片后调用,至少调用一次;
  • \n
  • 作用:关闭资源;
  • \n
  • 注意事项:可重复调用,关闭资源做非null检查,因为程序遇到异常情况可能直接跳转到closeInternal;
  • \n
\n

\n

openInputFormat

\n
    \n
  • 调用位置:创建分片之后调用;
  • \n
  • 作用:对整个InpurFormat资源做初始化;
  • \n
  • 注意事项:无;
  • \n
\n

\n

closeInputFormat

\n
    \n
  • 调用位置:当所有切片都执行完之后调用;
  • \n
  • 作用:关闭整个InputFormat的资源;
  • \n
  • 注意事项:无;
  • \n
\n

\n

OutputFormatBuilder

\n

需继承RichOutputFormatBuilder,和InputFormatBuilder相似

\n
public class SomeOutputFormatBuilder extends RichOutputFormatBuilder {\r\n    /**\r\n    * 首先实例化一个OutputFormat实例,通过构造函数传递,通过设计set方法设置参数\r\n    * 如下演示\r\n    */\r\n    protected SomeOutputFormat format;\r\n    \r\n    public SomeOutputFormatBuilder(SomeOutputFormat format){\r\n        super.format = this.format = format;\r\n    }\r\n    \r\n    public void setOneParameter(String oneParameter){\r\n        this.oneParameter = oneParameter;\r\n    }\r\n    \r\n    //重写checkFormat,检查参数设置是否正确\r\n    @Override\r\n    protected void checkFormat() {}\r\n}\n
\n

\n

OutputFormat

\n

需继承RichOutputFormat

\n
public class SomeOutputFormat extends RichOutputFormat {\r\n \t@Override\r\n    protected void openInternal(int taskNumber, int numTasks) throws IOException {}\r\n    \r\n    @Override\r\n    protected void writeSingleRecordInternal(Record record) {\r\n    }\r\n    \r\n    @Override\r\n    protected void writeMultipleRecordsInternal() throws Exception {\r\n    }\r\n}\n
\n

各方法的执行逻辑如下:

\n

openInternal -> writeSingleRecordInternal / writeMultipleRecordsInternal

\n

对于是执行writeSingleRecordInternal 还是writeMultipleRecordsInternal,关键参数是batchInterval,当batchInterval=1 时,框架调用writeSingleRecordInternal;当batchInterval > 1 且 record != null时,则调用writeMultipleRecordsInternal

\n

方法功能如下:\r\n

\n

openInternal

\n
    \n
  • 调用位置:开始写入使用
  • \n
  • 作用:用于打开需要读取的数据源,并做一些初始化;
  • \n
  • 注意事项:无;
  • \n
\n

\n

writerSingleRecordInternal

\n
    \n
  • 调用位置:openInernal之后调用,开始写入数据
  • \n
  • 作用:向数据源写入一条数据
  • \n
  • 注意事项:无;
  • \n
\n

\n

writerMultipleRecordsInternal

\n
    \n
  • 调用位置:openInternal之后调用,开始写入多条数据
  • \n
  • 作用:向数据源写入多条数据,由batchInterval参数决定写入多少条
  • \n
  • 注意事项:无;
  • \n
\n

\n

ChunJun数据结构

\n

ChunJun延续了Flink原生的数据类型Row

\n
@PublicEvolving\r\npublic class Row implements Serializable{\r\n\r\n\tprivate static final long serialVersionUID = 1L;\r\n\r\n\t/** The array to store actual values. */\r\n\tprivate final Object[] fields;\r\n\r\n\t/**\r\n\t * Create a new Row instance.\r\n\t * @param arity The number of fields in the Row\r\n\t */\r\n\tpublic Row(int arity) {\r\n\t\tthis.fields = new Object[arity];\r\n\t}\r\n}\n
\n

\n

任务json配置

\n

配置中尽量减少不必要的参数,有些参数框架中已有默认值,配置文件中的值优先,模板如下

\n
{\r\n  \"job\": {\r\n    \"content\": [\r\n      {\r\n        \"reader\": {\r\n          \"parameter\": {\r\n            \"username\": \"\",\r\n            \"password\": \"\",\r\n            \"connection\": [{\r\n              \"jdbcUrl\": [\"\"],\r\n              \"table\": [\r\n                \"\"\r\n              ]\r\n            }],\r\n            \"column\": [{\r\n              \"name\": \"id\",\r\n              \"type\": \"int\"\r\n            },{\r\n              \"name\":\"name\",\r\n              \"type\":\"string\"\r\n            }]\r\n          },\r\n          \"name\": \"mysqlreader\"\r\n        },\r\n        \"writer\": {\r\n          \"parameter\": {\r\n            \"print\": true\r\n          },\r\n          \"name\": \"streamwriter\"\r\n        }\r\n      }\r\n    ],\r\n    \"setting\": {\r\n      \"errorLimit\": {\r\n      },\r\n      \"speed\": {\r\n      }\r\n    }\r\n  }\r\n}\n
\n

\n

如何设计配置参数

\n

任务配置中readerwriterparameter部分是插件的配置参数,插件的配置参数应当遵循以下原则:

\n
    \n
  • 驼峰命名:所有配置项采用驼峰命名法,首字母小写,单词首字母大写。
  • \n
  • 正交原则:配置项必须正交,功能没有重复,没有潜规则。
  • \n
  • 富类型:合理使用json的类型,减少无谓的处理逻辑,减少出错的可能。\n
      \n
    • 使用正确的数据类型。比如,bool类型的值使用true/false,而非\"yes\"/\"true\"/0等。
    • \n
    • 合理使用集合类型,比如,用数组替代有分隔符的字符串。
    • \n
    \n
  • \n
  • 类似通用:遵守同一类型的插件的习惯,比如关系型数据库的connection参数都是如下结构:
  • \n
\n
{\r\n  \"connection\": [\r\n    {\r\n      \"table\": [\r\n        \"table_1\",\r\n        \"table_2\"\r\n      ],\r\n      \"jdbcUrl\": [\r\n        \"jdbc:mysql://127.0.0.1:3306/database_1\",\r\n        \"jdbc:mysql://127.0.0.2:3306/database_1_slave\"\r\n      ]\r\n    },\r\n    {\r\n      \"table\": [\r\n        \"table_3\",\r\n        \"table_4\"\r\n      ],\r\n      \"jdbcUrl\": [\r\n        \"jdbc:mysql://127.0.0.3:3306/database_2\",\r\n        \"jdbc:mysql://127.0.0.4:3306/database_2_slave\"\r\n      ]\r\n    }\r\n  ]\r\n}\n
\n

\n

如何处理脏数据

\n

\n

脏数据定义

\n
    \n
  1. Reader读到不支持的类型、不合法的值。
  2. \n
  3. 不支持的类型转换,比如:Bytes转换为Date
  4. \n
  5. 写入目标端失败,比如:写mysql整型长度超长。
  6. \n
\n

\n

如何处理脏数据

\n

框架会将脏数据临时存放起来。由DirtyDataManager实例写入临时存放脏数据文件中。

\n
    \n
  • path: 脏数据存放路径
  • \n
  • hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置)
  • \n
\n

\n

加载原理

\n
    \n
  1. 框架扫描plugin/readerplugin/writer目录,加载每个插件的plugin.json文件。
  2. \n
  3. plugin.json文件中name为key,索引所有的插件配置。如果发现重名的插件或者不存在的插件,框架会异常退出。
  4. \n
  5. 用户在插件中在reader/writer配置的name字段指定插件名字。框架根据插件的类型(reader/writer)和插件名称去插件的路径下扫描所有的jar,加入classpath
  6. \n
  7. 根据插件配置中定义的入口类,框架通过反射实例化对应的Job对象。
  8. \n
\n

\n

统一的目录结构

\n

\n

项目目录层级

\n

注意,插件Reader/Writer类需放在符合插件包名命名规则的reader下,如MysqlReader类需放在com.dtstack.chunjun.mysql.reader包下,具体命名规则参照 项目命名规则 内容

\n
\n

${CHUNJUN_HOME}\r\n|-- bin
\n| -- flink\r\n| -- chunjun.sh\r\n|\r\n|-- chunjun-somePlugin\r\n|-- chunjun-somePlugin-core\r\n|-- common 一些插件共用的类\r\n|-- exception 异常处理类\r\n|-- pom.xml 插件公用依赖\r\n|-- chunjun-somePlugin-reader\r\n|-- InputFormat\r\n|-- SomePluginInputFormat\r\n|-- SomePluginInputFormatBuiler\r\n|-- reader\r\n|-- SomePluginReader\r\n|-- chunjun-somePlugin-writer\r\n|-- OutputFormat\r\n|-- SomePluginOutputFormat\r\n|-- SomePluginOutputFormatBuiler\r\n|-- reader\r\n|-- SomePluginWriter

\n
\n

\n

项目命名规则

\n
    \n
  • 插件命名模板 [chunjun]-[dataSourceName],例如chunjun-mysql
  • \n
  • 插件模块命名模板 [chunjun]-[dataSourceName]-[reader/writer/core],例如chunjun-mysql-reader,chunjun-redis-writer
  • \n
  • 插件包名命名模板 [com.dtstack.chunjun.dataSource.xxxx],例如com.dtstack.chunjun.mysql.reader,com.dtstack.chunjun.redis.inputformat
  • \n
  • 插件Reader/Writer类命名模板 [dataSource][Reader/Writer],例如MysqlReader,RedisWriter,需注意,类似RestAPIWriter,MetaDataHive2Reader这样的命名是错误的,需改为RestapiWriter,Metadatahive2Reader
  • \n
\n

\n

插件打包

\n

进入项目根目录,使用maven打包:

\n

windows平台

\n
mvn clean package -DskipTests -Prelease -DscriptType=bat\n
\n

unix平台

\n
mvn clean package -DskipTests -Prelease -DscriptType=sh\n
\n

打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含ChunJun的启动脚本,plugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦!

","id":"5da3deb7-fc6b-5a02-b9f2-cf4291b1daf2","parent":{"id":"5e144293-a770-5f40-8373-f3fbd007d2a6","name":"contribution","modifiedTime":"2022-06-07T13:14:11.975Z","ino":3377699721739052}}},"pageContext":{"id":"5da3deb7-fc6b-5a02-b9f2-cf4291b1daf2","parent__name":"contribution","__params":{"parent__name":"contribution"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/db67bfa8-2060-581d-b9bd-9c1ff1c22ae8/page-data.json b/website/public/page-data/documents/db67bfa8-2060-581d-b9bd-9c1ff1c22ae8/page-data.json new file mode 100644 index 0000000000..6bc72678ed --- /dev/null +++ b/website/public/page-data/documents/db67bfa8-2060-581d-b9bd-9c1ff1c22ae8/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/db67bfa8-2060-581d-b9bd-9c1ff1c22ae8/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

一、Introduce

\n

The ElasticSearch Source plugin supports reading data from an existing ElasticSearch cluster in a specified index.

\n

二、Version support

\n

Elasticsearch 5.x\r\n​

\n

三、Plugin name

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
typename
Syncelasticsearch5reader
SQLelasticsearch5-x
\n

\n

四、Param description

\n

1、Sync

\n
    \n
  • hosts\n
      \n
    • Description:One or more Elasticsearch hosts to connect to。eg: [\"localhost:9200\"]
    • \n
    • Required:required
    • \n
    • Type:List
    • \n
    • Default:none
    • \n
    \n
  • \n
  • index\n
      \n
    • Description:Elasticsearch index for every record.
    • \n
    • Required:required
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • type\n
      \n
    • Description:Elasticsearch document type.
    • \n
    • Required:required
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • cluster\n
      \n
    • Description:name of connected es's cluster.
    • \n
    • Required:required
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • username\n
      \n
    • Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster.
    • \n
    • Required:optional
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • password\n
      \n
    • Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well.
    • \n
    • Required:optional
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • batchSize\n
      \n
    • Description:Number of data pieces written in batches
    • \n
    • Required:optional
    • \n
    • Type:Integer
    • \n
    • Default:1
    • \n
    \n
  • \n
  • column\n
      \n
    • Description:Columns to be synchronized
    • \n
    • note:'*' is not supported.
    • \n
    • format:
    • \n
    \n
  • \n
\n
\"column\": [{\r\n    \"name\": \"col\", -- Column name, which can be found in a multi-level format\r\n    \"type\": \"string\", -- Column type, when name is not specified, returns a constant column with the value specified by value\r\n    \"value\": \"value\" -- Constant column value\r\n}]\n
\n

\n

2、SQL

\n
    \n
  • hosts\n
      \n
    • Description:One or more Elasticsearch hosts to connect to。eg: [\"localhost:9200\"]
    • \n
    • Required:required
    • \n
    • Type:List
    • \n
    • Default:none
    • \n
    \n
  • \n
  • index\n
      \n
    • Description:Elasticsearch index for every record.
    • \n
    • Required:required
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • username\n
      \n
    • Description:User name after basic authentication is enabled. Please notice that Elasticsearch doesn't pre-bundled security feature, but you can enable it by following the guideline to secure an Elasticsearch cluster.
    • \n
    • Required:optional
    • \n
    • Type:String
    • \n
    • Default:none
    • \n
    \n
  • \n
  • password\n
      \n
    • Description:Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well.
    • \n
    • Required:optional
    • \n
    • Type:String
    • \n
    • Default:无
    • \n
    \n
  • \n
  • bulk-flush.max-actions\n
      \n
    • Description:Maximum number of actions to buffer for each bulk request.
    • \n
    • Required:否
    • \n
    • Type:Integer
    • \n
    • Default:1000
    • \n
    \n
  • \n
\n

\n

五、Data type

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
supporteddate type
yesINTEGER,SMALLINT,DECIMAL,TIMESTAM DOUBLE,FLOAT,DATE,VARCHAR,VARCHAR,TIMESTAMP,TIME,BYTE
noIP,binary, nested, object
\n

六、Sample demo

\n

See the 'demo' folder in the 'ChunJun: Local: Test' module of the project.

","id":"be686a9f-b88f-5a25-aff0-2cde9d6a25a9","parent":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","modifiedTime":"2022-06-15T14:45:51.160Z","ino":562949955565952}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"be686a9f-b88f-5a25-aff0-2cde9d6a25a9","parent__id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","__params":{"parent__id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/f56e6cbb-9ce1-5520-b121-de0a1f687954/page-data.json b/website/public/page-data/documents/f56e6cbb-9ce1-5520-b121-de0a1f687954/page-data.json new file mode 100644 index 0000000000..3457fe8a5f --- /dev/null +++ b/website/public/page-data/documents/f56e6cbb-9ce1-5520-b121-de0a1f687954/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-id-js","path":"/documents/f56e6cbb-9ce1-5520-b121-de0a1f687954/","result":{"data":{"markdownRemark":{"tableOfContents":"","html":"

插件通用配置

\n

一、Sync

\n

配置文件

\n

一个完整的ChunJun任务脚本配置包含 content,setting两个部分。content用于配置任务的输入源、输出源以及数据转换规则,其中包含reader,writer,transformer。setting则配置任务整体的环境设定,其中包含speed,errorLimit,dirty。具体如下所示:

\n
{\r\n  \"job\": {\r\n    \"content\": [\r\n      {\r\n        \"reader\": {},\r\n        \"writer\": {},\r\n        \"transformer\": {}\r\n      }\r\n    ],\r\n    \"setting\": {\r\n      \"speed\": {},\r\n      \"errorLimit\": {},\r\n      \"dirty\": {}\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
contentreaderreader插件详细配置
writerwriter插件详细配置
transformer数据转换SQL
settingspeed速率限制
errorLimit出错控制
dirty脏数据保存
\n

content配置

\n

reader

\n

reader用于配置数据的输入源,即数据从何而来。具体配置如下所示:

\n
{\r\n\"reader\" : {\r\n  \"name\" : \"xxreader\",\r\n  \"parameter\" : {},\r\n  \"table\": {\r\n    \"tableName\": \"xxx\"\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
namereader插件名称,具体名称参考各数据源配置文档
parameter数据源配置参数,具体配置参考各数据源配置文档
tableSQL源表名称开启transformer后必填
\n

writer

\n

writer用于配置数据的输出源,即数据写往何处。具体配置如下所示:

\n
{\r\n\"writer\" : {\r\n  \"name\" : \"xxwriter\",\r\n  \"parameter\" : {},\r\n  \"table\": {\r\n    \"tableName\": \"xxx\"\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
namewriter插件名称,具体名称参考各数据源配置文档
parameter数据源配置参数,具体配置参考各数据源配置文档
tableSQL结果表名称开启transformer后必填
\n

transformer配置

\n

transformer用于配置数据转换SQL,支持所有Flink原生语法及Function

\n
{\r\n\"transformer\" : {\r\n  \"transformSql\": \"xxx\"\r\n  }\r\n}\n
\n

setting配置

\n

speed

\n

speed用于配置任务并发数及速率限制。具体配置如下所示:

\n
{\r\n\"speed\" : {\r\n  \"channel\": 1,\r\n  \"readerChannel\": 2,\r\n  \"writerChannel\": 2,\r\n  \"bytes\": 0\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
channel整体任务并行度1int
readerChannelsource并行度-1int
writerChannelsink并行度-1int
bytesbytes >0则表示开启任务限速0Long
\n

errorLimit

\n

errorLimit用于配置任务运行时数据读取写入的出错控制。具体配置如下所示:

\n
{\r\n\"errorLimit\" : {\r\n  \"record\": 100,\r\n  \"percentage\": 10.0\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
record错误阈值,当错误记录数超过此阈值时任务失败0int
percentage错误比例阈值,当错误记录比例超过此阈值时任务失败0.0Double
\n

\n

metricPluginConf用于配置任务运行时自定义指标持久化的方式。具体配置如下所示:

\n
{\r\n\"metricPluginConf\":{\r\n          \"pluginName\": \"mysql\",\r\n          \"pluginProp\": {\r\n            \"jdbcUrl\": \"jdbc:mysql://localhost:3306/ide?useUnicode=true&characterEncoding=utf-8\",\r\n            \"schema\": \"ide\",\r\n            \"table\": \"chunjun_metrics\",\r\n            \"username\": \"drpeco\",\r\n            \"password\": \"DT@Stack#123\"\r\n          }\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
pluginName持久化插件的名称prometheusString
pluginProp连接插件需要用到的参数配置Map
\n

dirty

\n

dirty用于配置脏数据的保存,通常与上文出错控制联合使用。具体配置如下所示:

\n
{\r\n\"dirty\" : {\r\n  \"path\" : \"xxx\",\r\n  \"hadoopConfig\" : {\r\n  }\r\n }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
path脏数据保存路径Sring
hadoopConfigHadoop相关配置K-V键值对
\n

参考模板如下:

\n
{\r\n\"dirty\" : {\r\n        \"path\" : \"/user/hive/warehouse/xx.db/xx\",\r\n        \"hadoopConfig\" : {\r\n          \"fs.default.name\": \"hdfs://0.0.0.0:9000\",\r\n          \"dfs.ha.namenodes.ns1\" : \"nn1,nn2\",\r\n          \"dfs.namenode.rpc-address.ns1.nn1\" : \"0.0.0.0:9000\",\r\n          \"dfs.namenode.rpc-address.ns1.nn2\" : \"0.0.0.1:9000\",\r\n          \"dfs.client.failover.proxy.provider.ns1\" : \"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider\",\r\n          \"dfs.nameservices\" : \"ns1\"\r\n        }\r\n  }\r\n}\n
\n

二、SQL

\n

参考Flink官方文档

","id":"5f27d8bd-3a44-5514-afda-f5ed31c53e7b","parent":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","modifiedTime":"2022-06-15T14:45:51.218Z","ino":562949955566111}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{"id":"5f27d8bd-3a44-5514-afda-f5ed31c53e7b","parent__id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","__params":{"parent__id":"f56e6cbb-9ce1-5520-b121-de0a1f687954"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/generalconfig/page-data.json b/website/public/page-data/documents/generalconfig/page-data.json new file mode 100644 index 0000000000..c01997767a --- /dev/null +++ b/website/public/page-data/documents/generalconfig/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/generalconfig/","result":{"data":{"markdownRemark":{"html":"

插件通用配置

\n

一、Sync

\n

配置文件

\n

一个完整的ChunJun任务脚本配置包含 content,setting两个部分。content用于配置任务的输入源、输出源以及数据转换规则,其中包含reader,writer,transformer。setting则配置任务整体的环境设定,其中包含speed,errorLimit,dirty。具体如下所示:

\n
{\r\n  \"job\": {\r\n    \"content\": [\r\n      {\r\n        \"reader\": {},\r\n        \"writer\": {},\r\n        \"transformer\": {}\r\n      }\r\n    ],\r\n    \"setting\": {\r\n      \"speed\": {},\r\n      \"errorLimit\": {},\r\n      \"dirty\": {}\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
contentreaderreader插件详细配置
writerwriter插件详细配置
transformer数据转换SQL
settingspeed速率限制
errorLimit出错控制
dirty脏数据保存
\n

content配置

\n

reader

\n

reader用于配置数据的输入源,即数据从何而来。具体配置如下所示:

\n
{\r\n\"reader\" : {\r\n  \"name\" : \"xxreader\",\r\n  \"parameter\" : {},\r\n  \"table\": {\r\n    \"tableName\": \"xxx\"\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
namereader插件名称,具体名称参考各数据源配置文档
parameter数据源配置参数,具体配置参考各数据源配置文档
tableSQL源表名称开启transformer后必填
\n

writer

\n

writer用于配置数据的输出源,即数据写往何处。具体配置如下所示:

\n
{\r\n\"writer\" : {\r\n  \"name\" : \"xxwriter\",\r\n  \"parameter\" : {},\r\n  \"table\": {\r\n    \"tableName\": \"xxx\"\r\n    }\r\n  }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填
namewriter插件名称,具体名称参考各数据源配置文档
parameter数据源配置参数,具体配置参考各数据源配置文档
tableSQL结果表名称开启transformer后必填
\n

transformer配置

\n

transformer用于配置数据转换SQL,支持所有Flink原生语法及Function

\n
{\r\n\"transformer\" : {\r\n  \"transformSql\": \"xxx\"\r\n  }\r\n}\n
\n

setting配置

\n

speed

\n

speed用于配置任务并发数及速率限制。具体配置如下所示:

\n
{\r\n\"speed\" : {\r\n  \"channel\": 1,\r\n  \"readerChannel\": 2,\r\n  \"writerChannel\": 2,\r\n  \"bytes\": 0\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
channel整体任务并行度1int
readerChannelsource并行度-1int
writerChannelsink并行度-1int
bytesbytes >0则表示开启任务限速0Long
\n

errorLimit

\n

errorLimit用于配置任务运行时数据读取写入的出错控制。具体配置如下所示:

\n
{\r\n\"errorLimit\" : {\r\n  \"record\": 100,\r\n  \"percentage\": 10.0\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
record错误阈值,当错误记录数超过此阈值时任务失败0int
percentage错误比例阈值,当错误记录比例超过此阈值时任务失败0.0Double
\n

\n

metricPluginConf用于配置任务运行时自定义指标持久化的方式。具体配置如下所示:

\n
{\r\n\"metricPluginConf\":{\r\n          \"pluginName\": \"mysql\",\r\n          \"pluginProp\": {\r\n            \"jdbcUrl\": \"jdbc:mysql://localhost:3306/ide?useUnicode=true&characterEncoding=utf-8\",\r\n            \"schema\": \"ide\",\r\n            \"table\": \"chunjun_metrics\",\r\n            \"username\": \"drpeco\",\r\n            \"password\": \"DT@Stack#123\"\r\n          }\r\n}\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
pluginName持久化插件的名称prometheusString
pluginProp连接插件需要用到的参数配置Map
\n

dirty

\n

dirty用于配置脏数据的保存,通常与上文出错控制联合使用。具体配置如下所示:

\n
{\r\n\"dirty\" : {\r\n  \"path\" : \"xxx\",\r\n  \"hadoopConfig\" : {\r\n  }\r\n }\r\n}\n
\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明是否必填默认值参数类型
path脏数据保存路径Sring
hadoopConfigHadoop相关配置K-V键值对
\n

参考模板如下:

\n
{\r\n\"dirty\" : {\r\n        \"path\" : \"/user/hive/warehouse/xx.db/xx\",\r\n        \"hadoopConfig\" : {\r\n          \"fs.default.name\": \"hdfs://0.0.0.0:9000\",\r\n          \"dfs.ha.namenodes.ns1\" : \"nn1,nn2\",\r\n          \"dfs.namenode.rpc-address.ns1.nn1\" : \"0.0.0.0:9000\",\r\n          \"dfs.namenode.rpc-address.ns1.nn2\" : \"0.0.0.1:9000\",\r\n          \"dfs.client.failover.proxy.provider.ns1\" : \"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider\",\r\n          \"dfs.nameservices\" : \"ns1\"\r\n        }\r\n  }\r\n}\n
\n

二、SQL

\n

参考Flink官方文档

","id":"15aef91f-91f4-52fa-80ba-00b409909473","parent":{"id":"4b71b2b0-5dda-554c-964a-4ef892388d00","name":"generalconfig","modifiedTime":"2022-06-07T13:14:11.975Z","ino":3377699721739053}}},"pageContext":{"id":"15aef91f-91f4-52fa-80ba-00b409909473","parent__name":"generalconfig","__params":{"parent__name":"generalconfig"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/kerberos/page-data.json b/website/public/page-data/documents/kerberos/page-data.json new file mode 100644 index 0000000000..ac80111501 --- /dev/null +++ b/website/public/page-data/documents/kerberos/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/kerberos/","result":{"data":{"markdownRemark":{"html":"

数据源开启Kerberos安全认证

\n

目前ChunJun的部分插件支持了kerberos认证,有Hive、Hbase、HDFS三个插件。

\n

1.Kerberos证书加载方式

\n

目前支持两种方式,一种是从本地加载,即任务运行的机器上对应的目录必须存在配置里指定的证书文件,另一种是从sftp服务器下载,需要配置sftp服务器的配置信息。

\n

使用本地配置示例:

\n
\"hbaseConfig\": {\r\n \"hbase.zookeeper.property.clientPort\": \"2181\",\r\n \"hbase.rootdir\": \"hdfs://ns1/hbase\",\r\n \"hbase.cluster.distributed\": \"true\",\r\n \"hbase.zookeeper.quorum\": \"host1,host2,host3\",\r\n \"zookeeper.znode.parent\": \"/hbase\",\r\n \"hbase.security.authentication\":\"Kerberos\",\r\n \"hbase.security.authorization\":true,\r\n \"hbase.master.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"hbase.master.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"java.security.krb5.conf\":\"krb5.conf\",\r\n \"useLocalFile\":true\r\n }\n
\n

从sftp下载配置示例:

\n
\"hbaseConfig\": {\r\n \"hbase.zookeeper.property.clientPort\": \"2181\",\r\n \"hbase.rootdir\": \"hdfs://ns1/hbase\",\r\n \"hbase.cluster.distributed\": \"true\",\r\n \"hbase.zookeeper.quorum\": \"host1,host2,host3\",\r\n \"zookeeper.znode.parent\": \"/hbase\",\r\n \"hbase.security.authentication\":\"Kerberos\",\r\n \"hbase.security.authorization\":true,\r\n \"hbase.master.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"hbase.master.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"remoteDir\":\"/sftp/chunjun/keytab/hbase\",\r\n \"sftp\":{\r\n     \"host\":\"127.0.0.1\",\r\n     \"port\":\"22\",\r\n     \"username\":\"\",\r\n     \"password\":\"\"\r\n }\r\n }\n
\n

从sftp下载时的查找顺序:

\n

1.在/sftp/chunjun/keytab/hbase目录下查找hbase.keytab文件,如果找不到则2

\n

2.假设任务运行在node1机器上,则在/sftp/chunjun/keytab/hbase/node1下找hbase.keytab文件,找不到则报错;

\n

2.各数据源的配置

\n

hbase

\n
\"hbaseConfig\": {\r\n \"hbase.zookeeper.property.clientPort\": \"2181\",\r\n \"hbase.rootdir\": \"hdfs://ns1/hbase\",\r\n \"hbase.cluster.distributed\": \"true\",\r\n \"hbase.zookeeper.quorum\": \"host1,host2,host3\",\r\n \"zookeeper.znode.parent\": \"/hbase\",\r\n \"hbase.security.authentication\":\"Kerberos\",\r\n \"hbase.security.authorization\":true,\r\n \"hbase.master.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"hbase.master.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.keytab.file\":\"hbase.keytab\",\r\n \"hbase.regionserver.kerberos.principal\":\"hbase/node1@TEST.COM\",\r\n \"java.security.krb5.conf\":\"krb5.conf\"\r\n }\n
\n

hive

\n
\"hadoopConf\":{\r\n \"dfs.ha.namenodes.ns1\": \"nn1,nn2\",\r\n \"dfs.namenode.rpc-address.ns1.nn2\": \"node03:9000\",\r\n \"dfs.client.failover.proxy.provider.ns1\": \"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider\",\r\n \"dfs.namenode.rpc-address.ns1.nn1\": \"node02:9000\",\r\n \"dfs.nameservices\": \"ns1\"\r\n \"hadoop.security.authorization\": \"true\",\r\n \"hadoop.security.authentication\": \"Kerberos\",\r\n \"dfs.namenode.kerberos.principal\": \"hdfs/_HOST@HADOOP.COM\",\r\n \"dfs.namenode.keytab.file\": \"hdfs.keytab\",\r\n \"java.security.krb5.conf\": \"krb5.conf\"\r\n}\n
\n

jdbcUrl格式:jdbc:hive2://127.0.0.1:10000/default;principal=hive/node1@HADOOP.COM

\n

hdfs

\n
\"hadoopConf\":{\r\n    \"dfs.ha.namenodes.ns1\": \"nn1,nn2\",\r\n \"dfs.namenode.rpc-address.ns1.nn2\": \"node03:9000\",\r\n \"dfs.client.failover.proxy.provider.ns1\": \"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider\",\r\n \"dfs.namenode.rpc-address.ns1.nn1\": \"node02:9000\",\r\n \"dfs.nameservices\": \"ns1\"\r\n    \"hadoop.security.authorization\": \"true\",\r\n    \"hadoop.security.authentication\": \"Kerberos\",\r\n    \"dfs.namenode.kerberos.principal\": \"hdfs/_HOST@HADOOP.COM\",\r\n    \"dfs.namenode.keytab.file\": \"hdfs.keytab\",\r\n    \"java.security.krb5.conf\": \"krb5.conf\"\r\n}\n
","id":"c851a301-d02b-5738-be46-86810798cf12","parent":{"id":"398bb1d9-fd1e-59b3-972b-68ef211bace6","name":"kerberos","modifiedTime":"2022-06-07T13:14:12.084Z","ino":3377699721739129}}},"pageContext":{"id":"c851a301-d02b-5738-be46-86810798cf12","parent__name":"kerberos","__params":{"parent__name":"kerberos"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/page-data.json b/website/public/page-data/documents/page-data.json new file mode 100644 index 0000000000..e9fc399a73 --- /dev/null +++ b/website/public/page-data/documents/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-index-js","path":"/documents/","result":{"data":{"markdownRemark":{"html":"

connector 共用

\n

NOTE:新版chunjun支持connector和flinkSql自带的connector共用。

\n

在chunjun中使用flinkSql的connector

\n
    \n
  • 本地调试\n
      \n
    • chunjun-local-test模块下将对应的flink connector的GAV拷贝到pom.xml文件中
    • \n
    • 将chunjun中的connector的GAV拷贝到pom.xml文件中(部分flink connector已经通过chunjun connector引入,如果是则上一步省略)
    • \n
    • 在LocalTest类中指定参数运行即可\r\nsql:\n
        -- {\"id\":100,\"name\":\"lb james阿道夫\",\"money\":293.899778,\"dateone\":\"2020-07-30 10:08:22\",\"age\":\"33\",\"datethree\":\"2020-07-30 10:08:22.123\",\"datesix\":\"2020-07-30 10:08:22.123456\",\"datenigth\":\"2020-07-30 10:08:22.123456789\",\"dtdate\":\"2020-07-30\",\"dttime\":\"10:08:22\"}\r\n  CREATE TABLE source_ods_fact_user_ippv \r\n  (\r\n      id INT\r\n      , name STRING\r\n      , money decimal\r\n      , dateone timestamp\r\n      , age bigint\r\n      , datethree timestamp\r\n      , datesix timestamp(6)\r\n      , datenigth timestamp(9)\r\n      , dtdate date\r\n      , dttime time\r\n  ) WITH (\r\n      'connector' = 'kafka'\r\n      ,'topic' = 'da'\r\n      ,'properties.bootstrap.servers' = 'localhost:9092'\r\n      ,'properties.group.id' = 'luna_g'\r\n      ,'scan.startup.mode' = 'earliest-offset'\r\n      ,'format' = 'json'\r\n      ,'json.timestamp-format.standard' = 'SQL'\r\n  );\r\n  \r\n  CREATE TABLE result_total_pvuv_min\r\n  (\r\n      id INT\r\n      , name STRING\r\n      , money decimal\r\n      , dateone timestamp\r\n      , age bigint\r\n      , datethree timestamp\r\n      , datesix timestamp(6)\r\n      , datenigth timestamp(9)\r\n      , dtdate date\r\n      , dttime time\r\n  ) WITH (\r\n      'connector' = 'stream-x'\r\n  );\r\n  INSERT INTO result_total_pvuv_min\r\n  SELECT *\r\n  from source_ods_fact_user_ippv;\n
      \npom:\n
      <dependency>\r\n    <groupId>org.apache.flink</groupId>\r\n    <artifactId>flink-connector-kafka_2.12</artifactId>\r\n    <version>1.12.2</version>\r\n    <exclusions>\r\n        <exclusion>\r\n        <artifactId>flink-core</artifactId>\r\n        <groupId>org.apache.flink</groupId>\r\n        </exclusion>\r\n    </exclusions>\r\n</dependency>\r\n<dependency>\r\n    <groupId>com.dtstack.chunjun</groupId>\r\n    <artifactId>chunjun-connector-stream</artifactId>\r\n    <version>1.12-SNAPSHOT</version>\r\n</dependency>\n
      \n
    • \n
    \n
  • \n
\n
\n
    \n
  • 服务器上运行:\n
      \n
    • \n

      1.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到chunjun/lib目录下,构建jobGraph使用

      \n
    • \n
    • \n

      2.将需要的connector的jar包,和其他(包括chunjun-core-feat_1.12_pluginMerge.jar,如果kafka中用到了json也需要将json的format jar包)拷贝到flink/lib下

      \n
    • \n
    • \n

      3.在在chunjun/lib下目录下执行命令:

      \n
        \n
      • local模式
      • \n
      \n
      java -cp  \"./*\" com.dtstack.chunjun.client.Launcher \\\r\n-mode local \\\r\n-jobType sql \\\r\n-jobName flink1.12_SPI \\\r\n-job /yourjobpath/sqlFile.sql \\\r\n-chunjunDistDir /chunjun/chunjun-dist\n
      \n

      \"conectorShare_local.png\"

      \n
        \n
      • yarnPer模式
      • \n
      \n
      java -cp  \"./*\" com.dtstack.chunjun.client.Launcher \\\r\n-mode yarnPer \\\r\n-jobType sql \\\r\n-jobName flink1.12_SPI \\\r\n-job /yourjobpath/sqlFile.sql \\\r\n-chunjunDistDir /chunjun/chunjun-dist \\\r\n-flinkConfDir /flink/conf \\\r\n-hadoopConfDir /hadoop/etc \\\r\n-flinkLibDir /flink/lib \\\r\n-queue default\n
      \n

      \"conectorShare_yarnPer.png\"\r\n\"conectorShare_yarn.png\"

      \n
    • \n
    \n
  • \n
\n

在flinkSql中使用chunjun的connector

\n
    \n
  • 本地调试\n
      \n
    • 在自己项目中将对应的flink connector的GAV拷贝到pom.xml文件中
    • \n
    • 将chunjun中的core和connector的GAV拷贝到pom.xml文件中(需要先deploy项目)
    • \n
    • 运行自己的任务
    • \n
    \nsql:\n
        -- {\"id\":100,\"name\":\"lb james阿道夫\",\"money\":293.899778,\"dateone\":\"2020-07-30 10:08:22\",\"age\":\"33\",\"datethree\":\"2020-07-30 10:08:22.123\",\"datesix\":\"2020-07-30 10:08:22.123456\",\"datenigth\":\"2020-07-30 10:08:22.123456789\",\"dtdate\":\"2020-07-30\",\"dttime\":\"10:08:22\"}\r\n    CREATE TABLE source_ods_fact_user_ippv \r\n    (\r\n        id INT\r\n        , name STRING\r\n        , money decimal\r\n        , dateone timestamp\r\n        , age bigint\r\n        , datethree timestamp\r\n        , datesix timestamp(6)\r\n        , datenigth timestamp(9)\r\n        , dtdate date\r\n        , dttime time\r\n    ) WITH (\r\n        'connector' = 'kafka'\r\n        ,'topic' = 'da'\r\n        ,'properties.bootstrap.servers' = 'localhost:9092'\r\n        ,'properties.group.id' = 'luna_g'\r\n        ,'scan.startup.mode' = 'earliest-offset'\r\n        ,'format' = 'json'\r\n        ,'json.timestamp-format.standard' = 'SQL'\r\n    );\r\n    \r\n    CREATE TABLE result_total_pvuv_min\r\n    (\r\n        id INT\r\n        , name STRING\r\n        , money decimal\r\n        , dateone timestamp\r\n        , age bigint\r\n        , datethree timestamp\r\n        , datesix timestamp(6)\r\n        , datenigth timestamp(9)\r\n        , dtdate date\r\n        , dttime time\r\n    ) WITH (\r\n        'connector' = 'stream-x'\r\n    );\r\n    INSERT INTO result_total_pvuv_min\r\n    SELECT *\r\n    from source_ods_fact_user_ippv;\n
    \npom:\n
        <dependency>\r\n        <groupId>org.apache.flink</groupId>\r\n        <artifactId>flink-sql-connector-kafka_2.12</artifactId>\r\n        <version>1.12.2</version>\r\n    </dependency>\r\n    <dependency>\r\n        <groupId>com.dtstack.chunjun</groupId>\r\n        <artifactId>chunjun-core</artifactId>\r\n        <version>1.12-SNAPSHOT</version>\r\n    </dependency>\r\n    <dependency>\r\n        <groupId>com.dtstack.chunjun</groupId>\r\n        <artifactId>chunjun-connector-stream</artifactId>\r\n        <version>1.12-SNAPSHOT</version>\r\n    </dependency>\n
    \n
  • \n
\n
\n
    \n
  • 服务器上运行:\n
      \n
    • 将flink需要的connector和chunjun的chunjun-core-feat_1.12_pluginMerge.jar包和对应connector的jar引入到自己项目的pom中,将项目打成fat包,提交任务即可。
    • \n
    \n
  • \n
","id":"588d4c6f-b60a-550f-804f-5102fe9f5beb","parent":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","modifiedTime":"2022-06-15T14:45:51.157Z","ino":562949955565944}},"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}},"pageContext":{}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/documents/prometheus/page-data.json b/website/public/page-data/documents/prometheus/page-data.json new file mode 100644 index 0000000000..5a366a52eb --- /dev/null +++ b/website/public/page-data/documents/prometheus/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/prometheus/","result":{"data":{"markdownRemark":{"html":"

使用 prometheus pushgateway 需要设置的 confProp 参数

\n
    \n
  • metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter
  • \n
  • metrics.reporter.promgateway.host: prometheus pushgateway的地址
  • \n
  • metrics.reporter.promgateway.port:prometheus pushgateway的端口
  • \n
  • metrics.reporter.promgateway.jobName: 实例名称
  • \n
  • metrics.reporter.promgateway.randomJobNameSuffix: 是否在实例名称后面添加随机字符串(默认:true)
  • \n
  • metrics.reporter.promgateway.deleteOnShutdown: 是否在停止的时候删除数据(默认false)
  • \n
","id":"9780ed20-8104-57dc-9042-c9daddf85216","parent":{"id":"dcc4739d-3e38-59d9-9369-a1522b782d3f","name":"prometheus","modifiedTime":"2022-06-07T13:14:12.084Z","ino":3377699721739130}}},"pageContext":{"id":"9780ed20-8104-57dc-9042-c9daddf85216","parent__name":"prometheus","__params":{"parent__name":"prometheus"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/questions/page-data.json b/website/public/page-data/documents/questions/page-data.json new file mode 100644 index 0000000000..aa701c61b7 --- /dev/null +++ b/website/public/page-data/documents/questions/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/questions/","result":{"data":{"markdownRemark":{"html":"

常见问题

\n

1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包

\n

解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装:

\n
## windows平台\r\n./install_jars.bat\r\n\r\n## unix平台\r\n./install_jars.sh\n
\n

2.ChunJun版本需要与Flink版本保持一致

\n

1.8_release版本对应flink1.8\r\n1.10_release版本对应flink1.10 版本\r\n不对应在standalone和yarn session模式提交时,会报错:\r\nCaused by: java.io.InvalidClassException: org.apache.flink.api.common.operators.ResourceSpec; incompatible types for field cpuCores

\n

3.移动ChunJun lib目录下的Launcher包后,任务启动报错:错误: 找不到或无法加载主类

\n

chunjun启动脚本里面找的是lib目录下的所有jar包,而移动后lib中含有其他的jar包,这些jar包没有主类,因此报错\r\n可以使用如下的命令运行:\r\njava -cp chunjun/lib/chunjun-client.jar com.dtstack.chunjun.launcher.Launcher -mode local -job /opt/flink/flink/deps/job/stream.json -chunjunDistDir chunjun/chunjun-dist

\n

4. 关于编译chunjun core报错Failed to read artifact descriptor for com.google.errorprone:javac-shaded

\n

报错信息:

\n
[ERROR] Failed to execute goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check (spotless-check) on project chunjun-core: \r\nExecution spotless-check of goal com.diffplug.spotless:spotless-maven-plugin:2.4.2:check failed: Unable to resolve dependencies: \r\nFailed to collect dependencies at com.google.googlejavaformat:google-java-format:jar:1.7 -> com.google.errorprone:javac-shaded:jar:9+181-r4173-1: \r\nFailed to read artifact descriptor for com.google.errorprone:javac-shaded:jar:9+181-r4173-1: Could not transfer artifact \r\ncom.google.errorprone:javac-shaded:pom:9+181-r4173-1 from/to aliyunmaven (https://maven.aliyun.com/repository/public): \r\nAccess denied to:https://maven.aliyun.com/repository/public/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.pom -> [Help 1]\n
\n

解决:

\n

https://repo1.maven.org/maven2/com/google/errorprone/javac-shaded/9+181-r4173-1/javac-shaded-9+181-r4173-1.jar\r\n从这个地址下载javac-shaded-9+181-r4173-1.jar, 临时放到chunjun根目录下jars目录里,然后在源码根目录下 执行安装依赖包命令如下:

\n
mvn install:install-file -DgroupId=com.google.errorprone -DartifactId=javac-shaded -Dversion=9+181-r4173-1 -Dpackaging=jar -Dfile=./jars/javac-shaded-9+181-r4173-1.jar\n
","id":"a4ba6b29-daff-5a02-b79a-8ab1e1d86f6e","parent":{"id":"a2a96f8e-78b4-5098-88be-3560c8713bca","name":"questions","modifiedTime":"2022-06-07T13:14:12.084Z","ino":3377699721739131}}},"pageContext":{"id":"a4ba6b29-daff-5a02-b79a-8ab1e1d86f6e","parent__name":"questions","__params":{"parent__name":"questions"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/quickstart/page-data.json b/website/public/page-data/documents/quickstart/page-data.json new file mode 100644 index 0000000000..e7fd40a64b --- /dev/null +++ b/website/public/page-data/documents/quickstart/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/quickstart/","result":{"data":{"markdownRemark":{"html":"

下载代码

\n

1.使用git工具把项目clone到本地

\n
git clone https://github.com/DTStack/chunjun.git\r\ncd chunjun\n
\n

编译插件

\n

在chunjun home目录下执行

\n
mvn clean package -DskipTests \n
\n

或者执行

\n
sh build/build.sh\n
\n

常见问题

\n

1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包

\n

解决办法:在$CHUNJUN_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装:

\n
## windows平台\r\n./$CHUNJUN_HOME/bin/install_jars.bat\r\n\r\n## unix平台\r\n./$CHUNJUN_HOME/bin/install_jars.sh\n
\n

运行任务

\n

NOTE:项目中的chunjun-examples模块下提供了大量 数据同步案例SQL案例

\n

数据同步任务

\n

首先准备要运行的任务json,这里以stream插件为例(chunjun-examples文件夹下有大量案例):

\n
{\r\n  \"job\": {\r\n    \"content\": [\r\n      {\r\n        \"reader\": {\r\n          \"parameter\": {\r\n            \"column\": [\r\n              {\r\n                \"name\": \"id\",\r\n                \"type\": \"id\"\r\n              },\r\n              {\r\n                \"name\": \"name\",\r\n                \"type\": \"string\"\r\n              },\r\n              {\r\n                \"name\": \"content\",\r\n                \"type\": \"string\"\r\n              }\r\n            ],\r\n            \"sliceRecordCount\": [\"30\"],\r\n            \"permitsPerSecond\": 1\r\n          },\r\n          \"table\": {\r\n            \"tableName\": \"sourceTable\"\r\n          },\r\n          \"name\": \"streamreader\"\r\n        },\r\n        \"writer\": {\r\n          \"parameter\": {\r\n            \"column\": [\r\n              {\r\n                \"name\": \"id\",\r\n                \"type\": \"id\"\r\n              },\r\n              {\r\n                \"name\": \"name\",\r\n                \"type\": \"string\"\r\n              },\r\n              {\r\n                \"name\": \"content\",\r\n                \"type\": \"timestamp\"\r\n              }\r\n            ],\r\n            \"print\": true\r\n          },\r\n          \"table\": {\r\n            \"tableName\": \"sinkTable\"\r\n          },\r\n          \"name\": \"streamwriter\"\r\n        },\r\n        \"transformer\": {\r\n          \"transformSql\": \"select id,name, NOW() from sourceTable where CHAR_LENGTH(name) < 50 and CHAR_LENGTH(content) < 50\"\r\n        }\r\n      }\r\n    ],\r\n    \"setting\": {\r\n      \"errorLimit\": {\r\n        \"record\": 100\r\n      },\r\n      \"speed\": {\r\n        \"bytes\": 0,\r\n        \"channel\": 1,\r\n        \"readerChannel\": 1,\r\n        \"writerChannel\": 1\r\n      }\r\n    }\r\n  }\r\n}\n
\n

flinksql任务

\n

NOTE:ChunJun和flinkSql connector共用

\r\n或者准备要运行的flinksql任务,这里以stream插件为例(chunjun-examples文件夹下有大量案例):

\n
CREATE TABLE source\r\n(\r\n    id        INT,\r\n    name      STRING,\r\n    money     DECIMAL(32, 2),\r\n    dateone   timestamp,\r\n    age       bigint,\r\n    datethree timestamp,\r\n    datesix   timestamp(6),\r\n    datenigth timestamp(9),\r\n    dtdate    date,\r\n    dttime    time\r\n) WITH (\r\n      'connector' = 'stream-x',\r\n      'number-of-rows' = '10', -- 输入条数,默认无限\r\n      'rows-per-second' = '1' -- 每秒输入条数,默认不限制\r\n      );\r\n\r\nCREATE TABLE sink\r\n(\r\n    id        INT,\r\n    name      STRING,\r\n    money     DECIMAL(32, 2),\r\n    dateone   timestamp,\r\n    age       bigint,\r\n    datethree timestamp,\r\n    datesix   timestamp(6),\r\n    datenigth timestamp(9),\r\n    dtdate    date,\r\n    dttime    time\r\n) WITH (\r\n      'connector' = 'stream-x',\r\n      'print' = 'true'\r\n      );\r\n\r\ninsert into sink\r\nselect *\r\nfrom source;\n
\n

Local模式运行任务

\n

命令模板:

\n
bin/start-chunjun \\\r\n\t-mode local \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist \n
\n

可以在flink-conf.yaml配置文件里配置端口:

\n
## web服务端口,不指定的话会随机生成一个\r\nrest.bind-port: 8888\n
\n

使用下面的命令运行任务:

\n
bin/start-chunjun \\\r\n\t-mode local \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist \n
\n

任务运行后可以通过8888端口访问flink界面查看任务运行情况:

\n
\r\n \r\n
\n

Standalone模式运行

\n

NOTE:将chunjun-dist目录拷贝到$FLINK_HOME/lib下,并修改$FLINK_HOME/conf/flink-conf.yml中的classloader为classloader.resolve-order: parent-first

\n

命令模板:

\n
bin/start-chunjun \\\r\n\t-mode standalone \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-flinkConfDir $FLINK_HOME/conf \\\r\n\t-confProp \"{\\\"flink.checkpoint.interval\\\":60000}\"\n
\n

首先启动flink集群:

\n
# flink集群默认端口是8081\r\n$FLINK_HOME/bin/start-cluster.sh\n
\n

通过8081端口检查集群是否启动成功

\n
\r\n \r\n
\n

把任务提交到集群上运行:

\n
./bin/start-chunjun \\\r\n\t-mode standalone \\\r\n\t-jobType sync \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-flinkConfDir $FLINK_HOME/conf\n
\n

在集群上查看任务运行情况

\n
\r\n \r\n
\n

以Yarn Session模式运行任务

\n

NOTE:可以先在现在chunjun-clients模块YarnSessionClientUtil类中启动一个session,然后修改$FLINK_HOME/conf/flink-conf.yml中的classloader为classloader.resolve-order: parent-first

\n

命令示例:

\n
bin/start-chunjun \\\r\n\t-mode yarn-session \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-flinkConfDir $FLINK_HOME/conf \\\r\n\t-hadoopConfDir $HADOOP_HOME/etc/hadoop \\\r\n\t-confProp \"{\\\"flink.checkpoint.interval\\\":60000}\"\n
\n

首先确保yarn集群是可用的,然后手动启动一个yarn session:

\n
$FLINK_HOME/bin/yarn-session.sh -n 1 -s 1 -jm 1024 -tm 1024\n
\n
\r\n \r\n
\n
\r\n \r\n
\n

把任务提交到这个yarn session上:

\n
bin/start-chunjun \\\r\n\t-mode yarn-session \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-flinkConfDir $FLINK_HOME/conf \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-hadoopConfDir $HADOOP_HOME/etc/hadoop\n
\n

然后在flink界面查看任务运行情况:

\n
\r\n \r\n
\n

以Yarn Perjob模式运行任务

\n

命令示例:

\n
bin/start-chunjun \\\r\n\t-mode yarn-per-job \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-flinkConfDir $FLINK_HOME/conf \\\r\n\t-hadoopConfDir $HADOOP_HOME/etc/hadoop \\\r\n\t-flinkLibDir $FLINK_HOME/lib \\\r\n\t-confProp \"{\\\"flink.checkpoint.interval\\\":60000,\\\"yarn.application.queue\\\":\\\"default\\\"}\" \\ \n
\n

首先确保yarn集群是可用的,启动一个Yarn Application运行任务:

\n
bin/start-chunjun \\\r\n\t-mode yarn-per-job \\\r\n\t-jobType sync \\\r\n\t-job chunjun-examples/json/stream/stream.json \\\r\n\t-chunjunDistDir chunjun-dist  \\\r\n\t-hadoopConfDir $HADOOP_HOME/etc/hadoop \\\r\n\t-flinkLibDir $FLINK_HOME/lib \\\n
\n

然后在集群上查看任务运行情况

\n
\r\n \r\n
\n
\r\n \r\n
\n

Kubernetes Session模式运行任务

\n

命令示例:

\n
bin/start-chunjun \\\r\n    -mode kubernetes-session \\\r\n    -jobType sync \\\r\n    -job chunjun-examples/json/stream/stream.json \\\r\n    -jobName kubernetes-job \\\r\n    -jobType sync \\\r\n    -chunjunDistDir chunjun-dist  \\\r\n    -flinkLibDir $FLINK_HOME/lib \\\r\n    -flinkConfDir $FLINK_HOME/conf \\\r\n    -confProp \"{\\\"kubernetes.config.file\\\":\\\"${kubernetes_config_path}\\\",\\\"kubernetes.cluster-id\\\":\\\"${cluster_id}\\\",\\\"kubernetes.namespace\\\":\\\"${namespace}\\\"}\"\n
\n

需要提前手动在kubernetes上启动kubernetes session

\n
$FLINK_HOME/bin/kubernetes-session.sh -Dkubernetes.cluster-id=flink-session-test -Dclassloader.resolve-order=parent-first -Dkubernetes.container.image=${image_name}\n
\n

注意:需要提前构建chunjun镜像\r\nchunjun镜像构建说明

\n

Kubernetes Application模式运行任务

\n

命令示例:

\n
bin/start-chunjun \\\r\n    -mode kubernetes-application \\\r\n    -jobType sync \\\r\n    -job chunjun-examples/json/stream/stream.json \\\r\n    -jobName kubernetes-job \\\r\n    -jobType sync \\\r\n    -chunjunDistDir chunjun-dist  \\\r\n    -remotePluginPath /opt/chunjun-dist \\\r\n    -pluginLoadMode classpath \\\r\n    -flinkLibDir $FLINK_HOME/lib \\\r\n    -flinkConfDir $FLINK_HOME/conf \\\r\n    -confProp \"{\\\"kubernetes.config.file\\\":\\\"${kubernetes_config_path}\\\",\\\"kubernetes.container.image\\\":\\\"${image_name}\\\",\\\"kubernetes.namespace\\\":\\\"${namespace}\\\"}\"\n
\n

注意:需要提前构建chunjun镜像\r\nchunjun镜像构建说明

\n

参数说明

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
名称说明可选值是否必填默认值
mode执行模式,也就是flink集群的工作模式1.local: 本地模式
2.standalone: 独立部署模式的flink集群
3.yarn-session: yarn-session模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称\"Flink session cluster\"
4.yarn-per-job: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称\"Flink per-job cluster\"
5.kubernetes-session: kubernetes session模式提交任务,需要提前在kubernetes上启动flink session
6.kubernetes-application: kubernetes run application模式提交任务
local
jobType任务类型1.sync:数据同步任务
2.sql:flinksql任务
job同步、flinksql任务描述文件的存放路径;该描述文件中使用json、sql存放任务信息
jobName任务名称Flink Job
chunjunDistDir插件根目录地址,也就是打包后产生的chunjun-dist目录。$CHUNJUN_HOME/chunjun-dist
flinkConfDirflink配置文件所在的目录(单机模式下不需要)$FLINK_HOME/conf$FLINK_HOME/conf
flinkLibDirflink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib$FLINK_HOME/lib$FLINK_HOME/lib
hadoopConfDirHadoop配置文件(包括hdfs和yarn)所在的目录$HADOOP_HOME/etc/hadoop$HADOOP_HOME/etc/hadoop
pluginLoadModeyarn session模式插件加载方式1.classpath:提交任务时不上传插件包,需要在yarn-node节点chunjun-dist目录下部署插件包,但任务启动速度较快,session模式建议使用
2.shipfile:提交任务时上传chunjun-dist目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境,yarnPer模式建议使用
shipfile
confPropflink官方所有配置参数
p自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102
","id":"ec58a4be-07e6-5912-9cd1-a8dde394fbd5","parent":{"id":"6a0edac9-7160-56be-80f7-b478ed896bc0","name":"quickstart","modifiedTime":"2022-06-07T13:14:12.085Z","ino":3377699721739132}}},"pageContext":{"id":"ec58a4be-07e6-5912-9cd1-a8dde394fbd5","parent__name":"quickstart","__params":{"parent__name":"quickstart"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/documents/statistics/page-data.json b/website/public/page-data/documents/statistics/page-data.json new file mode 100644 index 0000000000..4c36e4d91d --- /dev/null +++ b/website/public/page-data/documents/statistics/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-documents-markdown-remark-parent-file-name-js","path":"/documents/statistics/","result":{"data":{"markdownRemark":{"html":"

ChunJun使用了flink内置Accumulator和Metric来记录任务的一些统计指标:

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
指标名称含义
numRead累计读取数据条数
byteRead累计读取数据字节数
readDuration读取数据的总时间
numWrite累计写入数据条数
byteWrite累计写入数据字节数
writeDuration写入数据的总时间
nErrors累计错误记录数
nullErrors累计空指针错误记录数
duplicateErrors累计主键冲突错误记录数
conversionErrors累计类型转换错误记录数
otherErrors累计其它错误记录数
\n

获取统计指标的方式

\n

1.Local模式运行

\n

local模式运行时,任务结束后会在控制台打印这些指标:

\n
---------------------------------\r\nnumWrite                  |  100\r\nlast_write_num_0          |  0\r\nconversionErrors          |  0\r\nwriteDuration             |  12251\r\nnumRead                   |  100\r\nduplicateErrors           |  0\r\nsnapshotWrite             |  0\r\nreadDuration              |  12247\r\notherErrors               |  0\r\nbyteRead                  |  2329\r\nlast_write_location_0     |  0\r\nbyteWrite                 |  2329\r\nnullErrors                |  0\r\nnErrors                   |  0\r\n---------------------------------\n
\n

2.yarn模式运行

\n
2.1 通过Flink REST接口获取
\n

任务运行期间,可以通过Flink REST接口获取Accumulator数据,名称和上面给出的一致。

\n

api:http://host:8088/proxy/application_1569335225689_4172//jobs/d5582272d29ff38e10416a4043a86cad/accumulators

\n

返回数据示例:

\n
{\r\n    \"job-accumulators\": [],\r\n    \"user-task-accumulators\": [\r\n        {\r\n            \"name\": \"numWrite\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"last_write_num_0\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"conversionErrors\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"writeDuration\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"numRead\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"duplicateErrors\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"snapshotWrite\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"readDuration\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"otherErrors\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"byteRead\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"last_write_location_0\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"byteWrite\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"nullErrors\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        },\r\n        {\r\n            \"name\": \"nErrors\",\r\n            \"type\": \"LongCounter\",\r\n            \"value\": \"0\"\r\n        }\r\n    ],\r\n    \"serialized-user-task-accumulators\": {}\r\n}\n
\n
2.2 将指标输出到其它系统
\n

比如将指标输出到prometheus,在flink的配置文件里增加配置即可:

\n
metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter\r\nmetrics.reporter.promgateway.interval: 500 MILLISECONDS \r\nmetrics.reporter.promgateway.host: 127.0.0.1 \r\nmetrics.reporter.promgateway.port: 9091\r\nmetrics.reporter.promgateway.jobName: testjob\r\nmetrics.reporter.promgateway.randomJobNameSuffix: true\r\nmetrics.reporter.promgateway.deleteOnShutdown: false\n
\n

通过prometheus获取数据时的名称为:

\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n
ChunJun中指标名称prometheus中指标名称
numReadflink_taskmanager_job_task_operator_chunjun_byteRead
byteReadflink_taskmanager_job_task_operator_chunjun_byteRead
readDurationflink_taskmanager_job_task_operator_chunjun_readDuration
numWriteflink_taskmanager_job_task_operator_chunjun_numWrite
byteWriteflink_taskmanager_job_task_operator_chunjun_byteWrite
writeDurationflink_taskmanager_job_task_operator_chunjun_writeDuration
nErrorsflink_taskmanager_job_task_operator_chunjun_nErrors
nullErrorsflink_taskmanager_job_task_operator_chunjun_nullErrors
duplicateErrorsflink_taskmanager_job_task_operator_chunjun_duplicateErrors
conversionErrorsflink_taskmanager_job_task_operator_chunjun_conversionErrors
otherErrorsflink_taskmanager_job_task_operator_chunjun_otherErrors
","id":"2bb08454-0d28-523b-9449-57a1518a2017","parent":{"id":"8362817b-b412-513f-973b-c3c107e14604","name":"statistics","modifiedTime":"2022-06-07T13:14:12.087Z","ino":3377699721739134}}},"pageContext":{"id":"2bb08454-0d28-523b-9449-57a1518a2017","parent__name":"statistics","__params":{"parent__name":"statistics"}}},"staticQueryHashes":["527733040"]} \ No newline at end of file diff --git a/website/public/page-data/download/page-data.json b/website/public/page-data/download/page-data.json new file mode 100644 index 0000000000..a7ee119886 --- /dev/null +++ b/website/public/page-data/download/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-download-js","path":"/download/","result":{"pageContext":{}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/examples/5e1f992d-2e11-5435-9c26-e3c8a71df04d/page-data.json b/website/public/page-data/examples/5e1f992d-2e11-5435-9c26-e3c8a71df04d/page-data.json new file mode 100644 index 0000000000..f3aaf2ae3e --- /dev/null +++ b/website/public/page-data/examples/5e1f992d-2e11-5435-9c26-e3c8a71df04d/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-examples-json-content-parent-file-id-js","path":"/examples/5e1f992d-2e11-5435-9c26-e3c8a71df04d/","result":{"data":{"jsonContent":{"id":"f252297e-9288-5b36-a71b-08883eaa1305","content":"{\r\n \"name\": \"gatsby-starter-default\",\r\n \"private\": true,\r\n \"description\": \"A simple starter to get up and developing quickly with Gatsby\",\r\n \"version\": \"0.1.0\",\r\n \"author\": \"Kyle Mathews \",\r\n \"dependencies\": {\r\n \"@icon-park/react\": \"^1.3.5\",\r\n \"@mantine/core\": \"^4.2.5\",\r\n \"@mantine/hooks\": \"^4.2.5\",\r\n \"antd\": \"^4.21.0\",\r\n \"aos\": \"^2.3.4\",\r\n \"autoprefixer\": \"^10.4.7\",\r\n \"gatsby\": \"^4.14.0\",\r\n \"gatsby-plugin-gatsby-cloud\": \"^4.14.0\",\r\n \"gatsby-plugin-image\": \"^2.14.1\",\r\n \"gatsby-plugin-manifest\": \"^4.14.0\",\r\n \"gatsby-plugin-mantine\": \"^4.0.0\",\r\n \"gatsby-plugin-offline\": \"^5.14.0\",\r\n \"gatsby-plugin-postcss\": \"^5.16.0\",\r\n \"gatsby-plugin-react-helmet\": \"^5.14.0\",\r\n \"gatsby-plugin-sass\": \"^5.14.0\",\r\n \"gatsby-plugin-sharp\": \"^4.14.1\",\r\n \"gatsby-remark-autolink-headers\": \"^5.16.0\",\r\n \"gatsby-remark-table-of-contents\": \"^2.0.0\",\r\n \"gatsby-source-filesystem\": \"^4.14.0\",\r\n \"gatsby-transformer-json\": \"^4.15.0\",\r\n \"gatsby-transformer-plaintext\": \"^2.0.0\",\r\n \"gatsby-transformer-remark\": \"^5.14.0\",\r\n \"gatsby-transformer-sharp\": \"^4.14.0\",\r\n \"node-sass\": \"^6.0.1\",\r\n \"postcss\": \"^8.4.14\",\r\n \"prop-types\": \"^15.8.1\",\r\n \"react\": \"^17.0.1\",\r\n \"react-dom\": \"^17.0.1\",\r\n \"react-helmet\": \"^6.1.0\",\r\n \"react-json-view\": \"^1.21.3\",\r\n \"tailwindcss\": \"^3.1.0\"\r\n },\r\n \"devDependencies\": {\r\n \"prettier\": \"^2.6.2\"\r\n },\r\n \"keywords\": [\r\n \"gatsby\"\r\n ],\r\n \"license\": \"0BSD\",\r\n \"scripts\": {\r\n \"build\": \"gatsby build\",\r\n \"develop\": \"gatsby develop\",\r\n \"format\": \"prettier --write \\\"**/*.{js,jsx,ts,tsx,json,md,css}\\\"\",\r\n \"start\": \"gatsby develop\",\r\n \"serve\": \"gatsby serve\",\r\n \"clean\": \"gatsby clean\",\r\n \"test\": \"echo \\\"Write tests! -> https://gatsby.dev/unit-testing\\\" && exit 1\"\r\n },\r\n \"repository\": {\r\n \"type\": \"git\",\r\n \"url\": \"https://github.com/gatsbyjs/gatsby-starter-default\"\r\n },\r\n \"bugs\": {\r\n \"url\": \"https://github.com/gatsbyjs/gatsby/issues\"\r\n }\r\n}\r\n"}},"pageContext":{"id":"f252297e-9288-5b36-a71b-08883eaa1305","parent__id":"5e1f992d-2e11-5435-9c26-e3c8a71df04d","__params":{"parent__id":"5e1f992d-2e11-5435-9c26-e3c8a71df04d"}}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/examples/page-data.json b/website/public/page-data/examples/page-data.json new file mode 100644 index 0000000000..4aef061afc --- /dev/null +++ b/website/public/page-data/examples/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-examples-index-js","path":"/examples/","result":{"data":{"jsonContent":{"id":"f252297e-9288-5b36-a71b-08883eaa1305","content":"{\r\n \"name\": \"gatsby-starter-default\",\r\n \"private\": true,\r\n \"description\": \"A simple starter to get up and developing quickly with Gatsby\",\r\n \"version\": \"0.1.0\",\r\n \"author\": \"Kyle Mathews \",\r\n \"dependencies\": {\r\n \"@icon-park/react\": \"^1.3.5\",\r\n \"@mantine/core\": \"^4.2.5\",\r\n \"@mantine/hooks\": \"^4.2.5\",\r\n \"antd\": \"^4.21.0\",\r\n \"aos\": \"^2.3.4\",\r\n \"autoprefixer\": \"^10.4.7\",\r\n \"gatsby\": \"^4.14.0\",\r\n \"gatsby-plugin-gatsby-cloud\": \"^4.14.0\",\r\n \"gatsby-plugin-image\": \"^2.14.1\",\r\n \"gatsby-plugin-manifest\": \"^4.14.0\",\r\n \"gatsby-plugin-mantine\": \"^4.0.0\",\r\n \"gatsby-plugin-offline\": \"^5.14.0\",\r\n \"gatsby-plugin-postcss\": \"^5.16.0\",\r\n \"gatsby-plugin-react-helmet\": \"^5.14.0\",\r\n \"gatsby-plugin-sass\": \"^5.14.0\",\r\n \"gatsby-plugin-sharp\": \"^4.14.1\",\r\n \"gatsby-remark-autolink-headers\": \"^5.16.0\",\r\n \"gatsby-remark-table-of-contents\": \"^2.0.0\",\r\n \"gatsby-source-filesystem\": \"^4.14.0\",\r\n \"gatsby-transformer-json\": \"^4.15.0\",\r\n \"gatsby-transformer-plaintext\": \"^2.0.0\",\r\n \"gatsby-transformer-remark\": \"^5.14.0\",\r\n \"gatsby-transformer-sharp\": \"^4.14.0\",\r\n \"node-sass\": \"^6.0.1\",\r\n \"postcss\": \"^8.4.14\",\r\n \"prop-types\": \"^15.8.1\",\r\n \"react\": \"^17.0.1\",\r\n \"react-dom\": \"^17.0.1\",\r\n \"react-helmet\": \"^6.1.0\",\r\n \"react-json-view\": \"^1.21.3\",\r\n \"tailwindcss\": \"^3.1.0\"\r\n },\r\n \"devDependencies\": {\r\n \"prettier\": \"^2.6.2\"\r\n },\r\n \"keywords\": [\r\n \"gatsby\"\r\n ],\r\n \"license\": \"0BSD\",\r\n \"scripts\": {\r\n \"build\": \"gatsby build\",\r\n \"develop\": \"gatsby develop\",\r\n \"format\": \"prettier --write \\\"**/*.{js,jsx,ts,tsx,json,md,css}\\\"\",\r\n \"start\": \"gatsby develop\",\r\n \"serve\": \"gatsby serve\",\r\n \"clean\": \"gatsby clean\",\r\n \"test\": \"echo \\\"Write tests! -> https://gatsby.dev/unit-testing\\\" && exit 1\"\r\n },\r\n \"repository\": {\r\n \"type\": \"git\",\r\n \"url\": \"https://github.com/gatsbyjs/gatsby-starter-default\"\r\n },\r\n \"bugs\": {\r\n \"url\": \"https://github.com/gatsbyjs/gatsby/issues\"\r\n }\r\n}\r\n"},"allFile":{"edges":[{"node":{"id":"5e1f992d-2e11-5435-9c26-e3c8a71df04d","name":"package","parent":null,"relativePath":"package.json","ctime":"2022-06-15T13:59:15.331Z","modifiedTime":"2022-06-15T13:59:15.331Z"}}]}},"pageContext":{}},"staticQueryHashes":[]} \ No newline at end of file diff --git a/website/public/page-data/index/page-data.json b/website/public/page-data/index/page-data.json new file mode 100644 index 0000000000..317e44de13 --- /dev/null +++ b/website/public/page-data/index/page-data.json @@ -0,0 +1 @@ +{"componentChunkName":"component---src-pages-index-js","path":"/","result":{"pageContext":{}},"staticQueryHashes":["63159454"]} \ No newline at end of file diff --git a/website/public/page-data/sq/d/1410458087.json b/website/public/page-data/sq/d/1410458087.json new file mode 100644 index 0000000000..beb88940c9 --- /dev/null +++ b/website/public/page-data/sq/d/1410458087.json @@ -0,0 +1 @@ +{"data":{"allFile":{"edges":[{"node":{"id":"5e1f992d-2e11-5435-9c26-e3c8a71df04d","name":"package","parent":null,"relativePath":"package.json","ctime":"2022-06-15T13:59:15.331Z","modifiedTime":"2022-06-15T13:59:15.331Z"}}]}}} \ No newline at end of file diff --git a/website/public/page-data/sq/d/3649515864.json b/website/public/page-data/sq/d/3649515864.json new file mode 100644 index 0000000000..95dcd8e42b --- /dev/null +++ b/website/public/page-data/sq/d/3649515864.json @@ -0,0 +1 @@ +{"data":{"site":{"siteMetadata":{"title":"Gatsby Default Starter"}}}} \ No newline at end of file diff --git a/website/public/page-data/sq/d/527733040.json b/website/public/page-data/sq/d/527733040.json new file mode 100644 index 0000000000..d3b33ffd0c --- /dev/null +++ b/website/public/page-data/sq/d/527733040.json @@ -0,0 +1 @@ +{"data":{"allFile":{"edges":[{"node":{"id":"5bed52d4-4216-529d-9f56-0b00e2f2f126","name":"conectorShare","parent":null,"relativePath":"conectorShare.md","ctime":"2022-06-15T14:45:51.157Z","modifiedTime":"2022-06-15T14:45:51.157Z"}},{"node":{"id":"c1b9081e-94fb-568e-9fd4-9f43308d1d1b","name":"confProp","parent":null,"relativePath":"confProp.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"2ea37140-1972-5012-8858-672bc7bd2751","name":"connectorShare","parent":null,"relativePath":"connectorShare.md","ctime":"2022-06-15T14:45:51.158Z","modifiedTime":"2022-06-15T14:45:51.158Z"}},{"node":{"id":"0eef8c51-10b1-5442-8a2e-9edaed9a90e8","name":"contribution","parent":null,"relativePath":"contribution.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"f56e6cbb-9ce1-5520-b121-de0a1f687954","name":"generalconfig","parent":null,"relativePath":"generalconfig.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"608e954e-8af3-5853-ac42-d86447b4d9a0","name":"iceberg","parent":null,"relativePath":"iceberg.md","ctime":"2022-06-15T14:45:51.219Z","modifiedTime":"2022-06-15T14:45:51.219Z"}},{"node":{"id":"bc8c3c68-ee04-5dbb-9c6e-0829cd4db17d","name":"kerberos","parent":null,"relativePath":"kerberos.md","ctime":"2022-06-15T14:45:51.317Z","modifiedTime":"2022-06-15T14:45:51.317Z"}},{"node":{"id":"1b353f9e-a434-50e5-bf3b-47e3ea51e808","name":"prometheus","parent":null,"relativePath":"prometheus.md","ctime":"2022-06-15T14:45:51.318Z","modifiedTime":"2022-06-15T14:45:51.318Z"}},{"node":{"id":"7a8f3b7e-4fea-5ec3-bb8f-d3fce2ca0fd7","name":"questions","parent":null,"relativePath":"questions.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"9aff3577-2c34-5d9f-be70-b6c633a7de4d","name":"quickstart","parent":null,"relativePath":"quickstart.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"b2e4dd0b-024a-5528-8281-a5e78ff075ce","name":"restore","parent":null,"relativePath":"restore.md","ctime":"2022-06-15T14:45:51.319Z","modifiedTime":"2022-06-15T14:45:51.319Z"}},{"node":{"id":"a948d2e4-10e5-52db-a2d3-003974ce0152","name":"statistics","parent":null,"relativePath":"statistics.md","ctime":"2022-06-15T14:45:51.320Z","modifiedTime":"2022-06-15T14:45:51.320Z"}},{"node":{"id":"db67bfa8-2060-581d-b9bd-9c1ff1c22ae8","name":"es5-source","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"b91682e2-8005-5e87-a719-354455ac7c82","name":"es5-sink","parent":null,"relativePath":"connectors-en/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.160Z","modifiedTime":"2022-06-15T14:45:51.160Z"}},{"node":{"id":"71f9fc5c-a31c-5609-942b-ccd842b13e9b","name":"es6-sink","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"694ee276-4040-55d5-bf94-dd3cdbc97ddf","name":"es6-lookup","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"30e430f1-c432-551a-b688-892134b2bfbc","name":"es6-source","parent":null,"relativePath":"connectors-en/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.161Z","modifiedTime":"2022-06-15T14:45:51.161Z"}},{"node":{"id":"f1ffa712-91ad-514e-b6e1-0b0eb3b064e5","name":"es7-lookup","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.162Z","modifiedTime":"2022-06-15T14:45:51.162Z"}},{"node":{"id":"3faa50be-d6b7-500a-9b45-6163966c05a9","name":"es7-sink","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"96216369-8952-51bc-9be6-6cf83f779728","name":"es7-source","parent":null,"relativePath":"connectors-en/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.163Z","modifiedTime":"2022-06-15T14:45:51.163Z"}},{"node":{"id":"ccc9d2bb-b6a4-5a39-944b-fba076e0aaee","name":"mongodb-lookup","parent":null,"relativePath":"connectors-en/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"0a1783f4-af55-5924-81c2-b48d052c3062","name":"binlog-source","parent":null,"relativePath":"connectors-en/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.159Z","modifiedTime":"2022-06-15T14:45:51.159Z"}},{"node":{"id":"5fb5df61-9d28-5cef-b9df-e0f47160132e","name":"mongodb-source","parent":null,"relativePath":"connectors-en/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"9915ccf0-2301-58a5-8d78-8e30d6a1a326","name":"mongodb-sink","parent":null,"relativePath":"connectors-en/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"b3959b7e-1e67-546b-9f7f-63b4f1793bea","name":"PostgresCDC-Source-en","parent":null,"relativePath":"connectors-en/pgwal/PostgresCDC-Source-en.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"10a8cc77-4fbd-5dcc-a5d7-cf9a5bc22178","name":"hdfs-sink-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-sink-en.md","ctime":"2022-06-15T14:45:51.164Z","modifiedTime":"2022-06-15T14:45:51.164Z"}},{"node":{"id":"3aa8db18-e48e-5fa9-b254-1ebb5832eec2","name":"hdfs-source-en","parent":null,"relativePath":"connectors-en/hdfs/hdfs-source-en.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"aaa33a66-9841-53ff-bcfc-b160dd7df1e2","name":"mysql-lookup","parent":null,"relativePath":"connectors-en/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.167Z","modifiedTime":"2022-06-15T14:45:51.167Z"}},{"node":{"id":"7f561cef-a143-564f-a824-7b9b92cb5d26","name":"oracle-lookup","parent":null,"relativePath":"connectors-en/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"20540b3e-20f7-5f3b-90ad-3a35718dd82f","name":"oracle-sink","parent":null,"relativePath":"connectors-en/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"058a4b17-b273-5387-8023-9d2005d6275b","name":"oracle-source","parent":null,"relativePath":"connectors-en/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.170Z","modifiedTime":"2022-06-15T14:45:51.170Z"}},{"node":{"id":"cfac3f8b-1332-5242-abe4-625a31e30c03","name":"mysql-sink","parent":null,"relativePath":"connectors-en/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.168Z","modifiedTime":"2022-06-15T14:45:51.168Z"}},{"node":{"id":"ba3c961e-85e2-5dd2-9281-2add6181b69d","name":"SqlserverCDC-configuration","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-configuration.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"eada14db-1e86-5680-b7ca-2680dc53c032","name":"mysql-source","parent":null,"relativePath":"connectors-en/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.169Z","modifiedTime":"2022-06-15T14:45:51.169Z"}},{"node":{"id":"1f18920f-09f1-54a6-a9af-35f26fbab9c0","name":"SqlserverCDC-Principles","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-Principles.md","ctime":"2022-06-15T14:45:51.173Z","modifiedTime":"2022-06-15T14:45:51.173Z"}},{"node":{"id":"f988eabe-e8d4-5a3f-acfd-953006152574","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors-en/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.174Z","modifiedTime":"2022-06-15T14:45:51.174Z"}},{"node":{"id":"90fcd642-5f3e-5604-968f-825b75f3b3ca","name":"sqlserver-lookup","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"5b07b05b-113b-58b1-9c50-57b41a227608","name":"sqlserver-source","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"4e0d7eaa-66e4-50e3-b0b3-d32c57445167","name":"redis-lookup","parent":null,"relativePath":"connectors-en/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"6e373bdf-2e53-57a0-9f4a-9ae3c19edd60","name":"redis-sink","parent":null,"relativePath":"connectors-en/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.171Z","modifiedTime":"2022-06-15T14:45:51.171Z"}},{"node":{"id":"b8e80fb9-b34a-5abc-a983-4490f9360f70","name":"kafka-source","parent":null,"relativePath":"connectors-en/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.166Z","modifiedTime":"2022-06-15T14:45:51.166Z"}},{"node":{"id":"a4f53b25-af66-5276-aefd-b19805120a07","name":"kafka-sink","parent":null,"relativePath":"connectors-en/kafka/kafka-sink.md","ctime":"2022-06-15T14:45:51.165Z","modifiedTime":"2022-06-15T14:45:51.165Z"}},{"node":{"id":"72c7a813-5a75-5872-a575-23c527f449e7","name":"sqlserver-sink","parent":null,"relativePath":"connectors-en/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.172Z","modifiedTime":"2022-06-15T14:45:51.172Z"}},{"node":{"id":"55326c3f-6bc7-5564-8de8-e8c3b42088c7","name":"binlog-source","parent":null,"relativePath":"connectors/binlog/binlog-source.md","ctime":"2022-06-15T14:45:51.175Z","modifiedTime":"2022-06-15T14:45:51.175Z"}},{"node":{"id":"7e3ba980-9692-5a2e-95f4-4f950f8de089","name":"db2-lookup","parent":null,"relativePath":"connectors/db2/db2-lookup.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"06f14c0d-0600-5987-9274-f8087c4077b1","name":"cassandra-lookup","parent":null,"relativePath":"connectors/cassandra/cassandra-lookup.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"ce4ddee7-ae18-5193-89cf-85fdfe69276a","name":"cassandra-sink","parent":null,"relativePath":"connectors/cassandra/cassandra-sink.md","ctime":"2022-06-15T14:45:51.177Z","modifiedTime":"2022-06-15T14:45:51.177Z"}},{"node":{"id":"0b2dac85-2a78-542f-bfae-464b57d4bb10","name":"db2-sink","parent":null,"relativePath":"connectors/db2/db2-sink.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"50096c33-6443-5aee-bc29-e3bdbcdac851","name":"db2-source","parent":null,"relativePath":"connectors/db2/db2-source.md","ctime":"2022-06-15T14:45:51.182Z","modifiedTime":"2022-06-15T14:45:51.182Z"}},{"node":{"id":"ceab5595-2cad-56c9-a664-5ed386762829","name":"dorisbatch-sink","parent":null,"relativePath":"connectors/doris/dorisbatch-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"54bdd64a-667d-5236-9b0e-0832c34debf3","name":"cassandra-source","parent":null,"relativePath":"connectors/cassandra/cassandra-source.md","ctime":"2022-06-15T14:45:51.178Z","modifiedTime":"2022-06-15T14:45:51.178Z"}},{"node":{"id":"31015037-3137-5747-870a-d796b9f1deb0","name":"es5-sink","parent":null,"relativePath":"connectors/elasticsearch5/es5-sink.md","ctime":"2022-06-15T14:45:51.183Z","modifiedTime":"2022-06-15T14:45:51.183Z"}},{"node":{"id":"88907719-b390-555d-b5f8-a0d5e481930b","name":"es6-sink","parent":null,"relativePath":"connectors/elasticsearch6/es6-sink.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5e4ac494-08ea-51c1-b146-42b6cff1f660","name":"es6-source","parent":null,"relativePath":"connectors/elasticsearch6/es6-source.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"4449c40d-e7c9-59b5-8e3d-d6661f0d2935","name":"es6-lookup","parent":null,"relativePath":"connectors/elasticsearch6/es6-lookup.md","ctime":"2022-06-15T14:45:51.185Z","modifiedTime":"2022-06-15T14:45:51.185Z"}},{"node":{"id":"5c57eb91-1ef2-52fe-a0ca-a2195efa04e3","name":"es5-source","parent":null,"relativePath":"connectors/elasticsearch5/es5-source.md","ctime":"2022-06-15T14:45:51.184Z","modifiedTime":"2022-06-15T14:45:51.184Z"}},{"node":{"id":"2a7e4f80-f7ff-586f-b3a9-56872ebb1bf0","name":"clickhouse-lookup","parent":null,"relativePath":"connectors/clickhouse/clickhouse-lookup.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"7cf28b4a-9eb1-50af-891a-589d659303db","name":"clickhouse-sink","parent":null,"relativePath":"connectors/clickhouse/clickhouse-sink.md","ctime":"2022-06-15T14:45:51.179Z","modifiedTime":"2022-06-15T14:45:51.179Z"}},{"node":{"id":"1cf60605-e0cf-5cfe-afc8-3a209fef3e70","name":"es7-lookup","parent":null,"relativePath":"connectors/elasticsearch7/es7-lookup.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"83e9d28d-6d36-5507-93c6-99319bd8dbca","name":"clickhouse-source","parent":null,"relativePath":"connectors/clickhouse/clickhouse-source.md","ctime":"2022-06-15T14:45:51.180Z","modifiedTime":"2022-06-15T14:45:51.180Z"}},{"node":{"id":"837f756f-4665-582e-8992-8a691b16ee4c","name":"es7-sink","parent":null,"relativePath":"connectors/elasticsearch7/es7-sink.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"7955d1c3-1727-597e-9407-1d489f6a66d8","name":"es7-source","parent":null,"relativePath":"connectors/elasticsearch7/es7-source.md","ctime":"2022-06-15T14:45:51.186Z","modifiedTime":"2022-06-15T14:45:51.186Z"}},{"node":{"id":"e0199b76-48f0-50c3-8da6-5d1e0c6e0529","name":"emqx-sink","parent":null,"relativePath":"connectors/emqx/emqx-sink.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"5f95d91c-7911-5590-8e0a-8d1951cc1ba1","name":"emqx-source","parent":null,"relativePath":"connectors/emqx/emqx-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"6a7a5091-fd5f-5161-a1eb-32b89a366830","name":"filesystem-sink","parent":null,"relativePath":"connectors/filesystem/filesystem-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"eef5105f-029b-54fd-97a2-f2b81a372c0f","name":"ftp-source","parent":null,"relativePath":"connectors/ftp/ftp-source.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"6015bcca-1157-561f-8290-3a5644d8e100","name":"file-source","parent":null,"relativePath":"connectors/file/file-source.md","ctime":"2022-06-15T14:45:51.187Z","modifiedTime":"2022-06-15T14:45:51.187Z"}},{"node":{"id":"e3c8934b-1838-54c5-8524-ca4bcc037c80","name":"gbase-lookup","parent":null,"relativePath":"connectors/gbase/gbase-lookup.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"c455aec8-9118-5f3f-953a-e909716d65f7","name":"ftp-sink","parent":null,"relativePath":"connectors/ftp/ftp-sink.md","ctime":"2022-06-15T14:45:51.188Z","modifiedTime":"2022-06-15T14:45:51.188Z"}},{"node":{"id":"32045ec3-89e9-5201-bff7-a73134393c7a","name":"greenplum-sink","parent":null,"relativePath":"connectors/greenplum/greenplum-sink.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"d41a4c9e-2d11-5a2d-940c-65341ad128e7","name":"gbase-sink","parent":null,"relativePath":"connectors/gbase/gbase-sink.md","ctime":"2022-06-15T14:45:51.189Z","modifiedTime":"2022-06-15T14:45:51.189Z"}},{"node":{"id":"a0c640d8-5228-5fdd-bd32-437b3ca35128","name":"gbase-source","parent":null,"relativePath":"connectors/gbase/gbase-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"a2461480-7278-523a-b2c7-8b473d83bb9a","name":"greenplum-source","parent":null,"relativePath":"connectors/greenplum/greenplum-source.md","ctime":"2022-06-15T14:45:51.190Z","modifiedTime":"2022-06-15T14:45:51.190Z"}},{"node":{"id":"960e8f11-d0a5-5985-b076-8fa7e0934d68","name":"hbase-lookup","parent":null,"relativePath":"connectors/hbase/hbase-lookup.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"f321424a-15d0-557a-9613-943c444e0866","name":"hdfs-sink","parent":null,"relativePath":"connectors/hdfs/hdfs-sink.md","ctime":"2022-06-15T14:45:51.193Z","modifiedTime":"2022-06-15T14:45:51.193Z"}},{"node":{"id":"bf2e998e-f34d-5659-a123-14fc9ffff5dc","name":"hdfs-source","parent":null,"relativePath":"connectors/hdfs/hdfs-source.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"a6da894c-8a3b-5adf-972c-ff77584d0469","name":"hive-lookup","parent":null,"relativePath":"connectors/hive/hive-lookup.md","ctime":"2022-06-15T14:45:51.194Z","modifiedTime":"2022-06-15T14:45:51.194Z"}},{"node":{"id":"0dbc98fd-cc18-5970-a3ae-7c5e5691c4d3","name":"hbase-sink","parent":null,"relativePath":"connectors/hbase/hbase-sink.md","ctime":"2022-06-15T14:45:51.191Z","modifiedTime":"2022-06-15T14:45:51.191Z"}},{"node":{"id":"026f8d50-34a3-58a3-9e9d-a9bff77581a2","name":"hbase-source","parent":null,"relativePath":"connectors/hbase/hbase-source.md","ctime":"2022-06-15T14:45:51.192Z","modifiedTime":"2022-06-15T14:45:51.192Z"}},{"node":{"id":"5b4c347c-ac8f-5bb0-8641-6bea56ca40bb","name":"hive-sink","parent":null,"relativePath":"connectors/hive/hive-sink.md","ctime":"2022-06-15T14:45:51.195Z","modifiedTime":"2022-06-15T14:45:51.195Z"}},{"node":{"id":"31e148cf-6bec-57fa-a270-3b6c38dd329b","name":"kafka-sink","parent":null,"relativePath":"connectors/kafka/kafka-sink.md","ctime":"2022-06-15T14:55:19.150Z","modifiedTime":"2022-06-15T14:55:19.150Z"}},{"node":{"id":"d002a094-a656-56c6-974a-78823e7c2ab9","name":"kafka-source","parent":null,"relativePath":"connectors/kafka/kafka-source.md","ctime":"2022-06-15T14:45:51.198Z","modifiedTime":"2022-06-15T14:45:51.198Z"}},{"node":{"id":"be159d5f-6e5c-580e-8a41-f8c28c97ac1e","name":"http-sink","parent":null,"relativePath":"connectors/http/http-sink.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"02fe41db-e7e5-59bf-a02b-b8877cdce82a","name":"http-source","parent":null,"relativePath":"connectors/http/http-source.md","ctime":"2022-06-15T14:45:51.196Z","modifiedTime":"2022-06-15T14:45:51.196Z"}},{"node":{"id":"bfb32f91-2c2a-5668-accf-524c1301e298","name":"kingbase-sink","parent":null,"relativePath":"connectors/kingbase/kingbase-sink.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"87d74717-3bc2-595e-a1e5-e05b6261aca0","name":"LogMiner-source","parent":null,"relativePath":"connectors/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.201Z","modifiedTime":"2022-06-15T14:45:51.201Z"}},{"node":{"id":"a64ec76c-df1f-5bed-959f-35fbf94861f8","name":"kingbase-source","parent":null,"relativePath":"connectors/kingbase/kingbase-source.md","ctime":"2022-06-15T14:45:51.199Z","modifiedTime":"2022-06-15T14:45:51.199Z"}},{"node":{"id":"56e03f9a-1b17-5bbb-9b48-28ca5e5501e2","name":"LogMiner原理","parent":null,"relativePath":"connectors/logminer/LogMiner原理.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"9fe6f140-7450-59d2-a740-ac6f20114a79","name":"LogMiner配置","parent":null,"relativePath":"connectors/logminer/LogMiner配置.md","ctime":"2022-06-15T14:45:51.202Z","modifiedTime":"2022-06-15T14:45:51.202Z"}},{"node":{"id":"c18c919c-e710-53ad-9550-af2d736a317a","name":"mongodb-lookup","parent":null,"relativePath":"connectors/mongodb/mongodb-lookup.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e429f8e7-c020-5f9b-9f97-1c3d5689f398","name":"kudu-lookup","parent":null,"relativePath":"connectors/kudu/kudu-lookup.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"7f6dcc82-ea76-5e94-8f65-ddb1b1493c00","name":"mongodb-sink","parent":null,"relativePath":"connectors/mongodb/mongodb-sink.md","ctime":"2022-06-15T14:45:51.203Z","modifiedTime":"2022-06-15T14:45:51.203Z"}},{"node":{"id":"e288ac0c-b263-5e39-9992-28211ec6ee16","name":"mongodb-source","parent":null,"relativePath":"connectors/mongodb/mongodb-source.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"485f9bc9-ef7d-5422-ba1f-5304f73e69a0","name":"kudu-sink","parent":null,"relativePath":"connectors/kudu/kudu-sink.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"ee45d04c-cce0-5be6-ac0f-d4b4e6e458a8","name":"kudu-source","parent":null,"relativePath":"connectors/kudu/kudu-source.md","ctime":"2022-06-15T14:45:51.200Z","modifiedTime":"2022-06-15T14:45:51.200Z"}},{"node":{"id":"1d154249-006b-528f-998e-53e7fc5158d3","name":"Postgres-CDC","parent":null,"relativePath":"connectors/pgwal/Postgres-CDC.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"69b00a40-e018-50d8-9274-7e3fb6eeee77","name":"redis-lookup","parent":null,"relativePath":"connectors/redis/redis-lookup.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"54a59cfe-7852-585a-a94b-0e8e0e0a4583","name":"redis-sink","parent":null,"relativePath":"connectors/redis/redis-sink.md","ctime":"2022-06-15T14:45:51.210Z","modifiedTime":"2022-06-15T14:45:51.210Z"}},{"node":{"id":"ba3c4136-99cd-5297-98ad-06db24d7efcd","name":"postgres-lookup","parent":null,"relativePath":"connectors/postgres/postgres-lookup.md","ctime":"2022-06-15T14:45:51.207Z","modifiedTime":"2022-06-15T14:45:51.207Z"}},{"node":{"id":"5b02999b-9dfb-5ac3-91a1-72641f88c615","name":"postgres-sink","parent":null,"relativePath":"connectors/postgres/postgres-sink.md","ctime":"2022-06-15T14:45:51.208Z","modifiedTime":"2022-06-15T14:45:51.208Z"}},{"node":{"id":"418bad96-fae0-5f3d-820e-05003683e769","name":"postgres-source","parent":null,"relativePath":"connectors/postgres/postgres-source.md","ctime":"2022-06-15T14:45:51.209Z","modifiedTime":"2022-06-15T14:45:51.209Z"}},{"node":{"id":"03423183-9241-5cc9-b5c2-ffab536af8e4","name":"mysql-lookup","parent":null,"relativePath":"connectors/mysql/mysql-lookup.md","ctime":"2022-06-15T14:45:51.204Z","modifiedTime":"2022-06-15T14:45:51.204Z"}},{"node":{"id":"46f9411a-0f08-510a-8827-1220e7f587c3","name":"mysql-sink","parent":null,"relativePath":"connectors/mysql/mysql-sink.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"c4436fca-f20f-5c03-97b7-e345a3b3c1ed","name":"mysql-source","parent":null,"relativePath":"connectors/mysql/mysql-source.md","ctime":"2022-06-15T14:45:51.205Z","modifiedTime":"2022-06-15T14:45:51.205Z"}},{"node":{"id":"84d7a592-8ece-5265-96a0-1d00ab970614","name":"saphana-sink","parent":null,"relativePath":"connectors/saphana/saphana-sink.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"5c87db99-dfd3-5d33-a363-d7648a555b49","name":"saphana-source","parent":null,"relativePath":"connectors/saphana/saphana-source.md","ctime":"2022-06-15T14:45:51.211Z","modifiedTime":"2022-06-15T14:45:51.211Z"}},{"node":{"id":"b3154ffd-5ef7-5136-a588-4aca15d33628","name":"solr-sink","parent":null,"relativePath":"connectors/solr/solr-sink.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ce5fc1f3-56f9-5ce8-8019-babcf30b0df2","name":"solr-source","parent":null,"relativePath":"connectors/solr/solr-source.md","ctime":"2022-06-15T14:45:51.212Z","modifiedTime":"2022-06-15T14:45:51.212Z"}},{"node":{"id":"ff49f003-a363-507c-80bd-fa62c7638d8f","name":"sqlserver-lookup","parent":null,"relativePath":"connectors/sqlserver/sqlserver-lookup.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"6522dffa-4022-5c82-bac5-546329d17960","name":"sqlserver-sink","parent":null,"relativePath":"connectors/sqlserver/sqlserver-sink.md","ctime":"2022-06-15T14:45:51.213Z","modifiedTime":"2022-06-15T14:45:51.213Z"}},{"node":{"id":"92442492-7fc5-5c30-8361-521a08d8dda6","name":"sqlserver-source","parent":null,"relativePath":"connectors/sqlserver/sqlserver-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"83464926-94af-5996-88a4-16a8c8577050","name":"oracle-lookup","parent":null,"relativePath":"connectors/oracle/oracle-lookup.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"45a3d0cc-1493-5f2d-b021-8e6e948270b6","name":"stream-sink","parent":null,"relativePath":"connectors/stream/stream-sink.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"efa70c0e-a44a-516e-a527-1116abfb7609","name":"oracle-sink","parent":null,"relativePath":"connectors/oracle/oracle-sink.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"7d3b4c50-3685-566d-a7b5-e8843eb83bff","name":"oracle-source","parent":null,"relativePath":"connectors/oracle/oracle-source.md","ctime":"2022-06-15T14:45:51.206Z","modifiedTime":"2022-06-15T14:45:51.206Z"}},{"node":{"id":"91832125-df25-5124-af00-e2bf78d678ff","name":"stream-source","parent":null,"relativePath":"connectors/stream/stream-source.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"a5ee8675-566d-5ecd-af50-911178bb7712","name":"starrocks-sink","parent":null,"relativePath":"connectors/starrocks/starrocks-sink.md","ctime":"2022-06-15T14:45:51.216Z","modifiedTime":"2022-06-15T14:45:51.216Z"}},{"node":{"id":"5024a8b0-ca47-5624-9325-8ee6742fd04c","name":"tidb-lookup","parent":null,"relativePath":"connectors/tidb/tidb-lookup.md","ctime":"2022-06-15T14:45:51.217Z","modifiedTime":"2022-06-15T14:45:51.217Z"}},{"node":{"id":"e5abf1ee-e88c-5c5c-b4e5-2f0bc9d2c3c7","name":"tidb-sink","parent":null,"relativePath":"connectors/tidb/tidb-sink.md","ctime":"2022-06-15T14:45:51.218Z","modifiedTime":"2022-06-15T14:45:51.218Z"}},{"node":{"id":"33492436-1819-561b-926e-ff329b246738","name":"SqlserverCDC-source","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC-source.md","ctime":"2022-06-15T14:45:51.214Z","modifiedTime":"2022-06-15T14:45:51.214Z"}},{"node":{"id":"50096837-25af-50f7-b187-11bb58fa4e87","name":"SqlserverCDC原理","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC原理.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"0b893e20-56cd-5ef0-931f-af5183104aa3","name":"SqlserverCDC配置","parent":null,"relativePath":"connectors/sqlservercdc/SqlserverCDC配置.md","ctime":"2022-06-15T14:45:51.215Z","modifiedTime":"2022-06-15T14:45:51.215Z"}},{"node":{"id":"ed763f9f-301a-5cbf-b387-5b22d4443884","name":"LogMiner-source","parent":null,"relativePath":"connectors/connectors-en/logminer/LogMiner-source.md","ctime":"2022-06-15T14:45:51.181Z","modifiedTime":"2022-06-15T14:45:51.181Z"}}]}}} \ No newline at end of file diff --git a/website/public/page-data/sq/d/63159454.json b/website/public/page-data/sq/d/63159454.json new file mode 100644 index 0000000000..392fa653d4 --- /dev/null +++ b/website/public/page-data/sq/d/63159454.json @@ -0,0 +1 @@ +{"data":{"site":{"siteMetadata":{"title":"Gatsby Default Starter","description":"Kick off your next, great Gatsby project with this default starter. This barebones starter ships with the main Gatsby configuration files you might need.","author":"@gatsbyjs"}}}} \ No newline at end of file diff --git a/website/public/~partytown/debug/partytown-atomics.js b/website/public/~partytown/debug/partytown-atomics.js new file mode 100644 index 0000000000..404e664590 --- /dev/null +++ b/website/public/~partytown/debug/partytown-atomics.js @@ -0,0 +1,536 @@ +/* Partytown 0.5.4 - MIT builder.io */ +(window => { + const isPromise = v => "object" == typeof v && v && v.then; + const noop = () => {}; + const len = obj => obj.length; + const getConstructorName = obj => { + var _a, _b, _c; + try { + const constructorName = null === (_a = null == obj ? void 0 : obj.constructor) || void 0 === _a ? void 0 : _a.name; + if (constructorName) { + return constructorName; + } + } catch (e) {} + try { + const zoneJsConstructorName = null === (_c = null === (_b = null == obj ? void 0 : obj.__zone_symbol__originalInstance) || void 0 === _b ? void 0 : _b.constructor) || void 0 === _c ? void 0 : _c.name; + if (zoneJsConstructorName) { + return zoneJsConstructorName; + } + } catch (e) {} + return ""; + }; + const startsWith = (str, val) => str.startsWith(val); + const isValidMemberName = memberName => !(startsWith(memberName, "webkit") || startsWith(memberName, "toJSON") || startsWith(memberName, "constructor") || startsWith(memberName, "toString") || startsWith(memberName, "_")); + const getNodeName = node => 11 === node.nodeType && node.host ? "#s" : node.nodeName; + const randomId = () => Math.round(Math.random() * Number.MAX_SAFE_INTEGER).toString(36); + const defineConstructorName = (Cstr, value) => ((obj, memberName, descriptor) => Object.defineProperty(obj, memberName, { + ...descriptor, + configurable: true + }))(Cstr, "name", { + value: value + }); + const htmlConstructorTags = { + Anchor: "a", + DList: "dl", + Image: "img", + OList: "ol", + Paragraph: "p", + Quote: "q", + TableCaption: "caption", + TableCell: "td", + TableCol: "colgroup", + TableRow: "tr", + TableSection: "tbody", + UList: "ul" + }; + const svgConstructorTags = { + Graphics: "g", + SVG: "svg" + }; + const InstanceIdKey = Symbol(); + const CreatedKey = Symbol(); + const instances = new Map; + const mainRefs = new Map; + const winCtxs = {}; + const windowIds = new WeakMap; + const getAndSetInstanceId = (instance, instanceId) => { + if (instance) { + if (instanceId = windowIds.get(instance)) { + return instanceId; + } + (instanceId = instance[InstanceIdKey]) || setInstanceId(instance, instanceId = randomId()); + return instanceId; + } + }; + const getInstance = (winId, instanceId, win, doc, docId) => { + if ((win = winCtxs[winId]) && win.$window$) { + if (winId === instanceId) { + return win.$window$; + } + doc = win.$window$.document; + docId = instanceId.split(".").pop(); + if ("d" === docId) { + return doc; + } + if ("e" === docId) { + return doc.documentElement; + } + if ("h" === docId) { + return doc.head; + } + if ("b" === docId) { + return doc.body; + } + } + return instances.get(instanceId); + }; + const setInstanceId = (instance, instanceId, now) => { + if (instance) { + instances.set(instanceId, instance); + instance[InstanceIdKey] = instanceId; + instance[CreatedKey] = now = Date.now(); + if (now > lastCleanup + 5e3) { + instances.forEach(((storedInstance, instanceId) => { + storedInstance[CreatedKey] < lastCleanup && storedInstance.nodeType && !storedInstance.isConnected && instances.delete(instanceId); + })); + lastCleanup = now; + } + } + }; + let lastCleanup = 0; + const mainWindow = window.parent; + const docImpl = document.implementation.createHTMLDocument(); + const config = mainWindow.partytown || {}; + const libPath = (config.lib || "/~partytown/") + "debug/"; + const logMain = msg => { + console.debug.apply(console, [ "%cMain 🌎", "background: #717171; color: white; padding: 2px 3px; border-radius: 2px; font-size: 0.8em;", msg ]); + }; + const winIds = []; + const normalizedWinId = winId => { + winIds.includes(winId) || winIds.push(winId); + return winIds.indexOf(winId) + 1; + }; + const defineCustomElement = (winId, worker, ceData) => { + const Cstr = defineConstructorName(class extends winCtxs[winId].$window$.HTMLElement {}, ceData[0]); + const ceCallbackMethods = "connectedCallback,disconnectedCallback,attributeChangedCallback,adoptedCallback".split(","); + ceCallbackMethods.map((callbackMethodName => Cstr.prototype[callbackMethodName] = function(...args) { + worker.postMessage([ 15, winId, getAndSetInstanceId(this), callbackMethodName, args ]); + })); + Cstr.observedAttributes = ceData[1]; + return Cstr; + }; + const serializeForWorker = ($winId$, value, added, type, cstrName) => void 0 !== value && (type = typeof value) ? "string" === type || "number" === type || "boolean" === type || null == value ? [ 0, value ] : "function" === type ? [ 6 ] : (added = added || new Set) && Array.isArray(value) ? added.has(value) ? [ 1, [] ] : added.add(value) && [ 1, value.map((v => serializeForWorker($winId$, v, added))) ] : "object" === type ? "" === (cstrName = getConstructorName(value)) ? [ 2, {} ] : "Window" === cstrName ? [ 3, [ $winId$, $winId$ ] ] : "HTMLCollection" === cstrName || "NodeList" === cstrName ? [ 7, Array.from(value).map((v => serializeForWorker($winId$, v, added)[1])) ] : cstrName.endsWith("Event") ? [ 5, serializeObjectForWorker($winId$, value, added) ] : "CSSRuleList" === cstrName ? [ 12, Array.from(value).map(serializeCssRuleForWorker) ] : startsWith(cstrName, "CSS") && cstrName.endsWith("Rule") ? [ 11, serializeCssRuleForWorker(value) ] : "CSSStyleDeclaration" === cstrName ? [ 13, serializeObjectForWorker($winId$, value, added) ] : "Attr" === cstrName ? [ 10, [ value.name, value.value ] ] : value.nodeType ? [ 3, [ $winId$, getAndSetInstanceId(value), getNodeName(value) ] ] : [ 2, serializeObjectForWorker($winId$, value, added, true, true) ] : void 0 : value; + const serializeObjectForWorker = (winId, obj, added, includeFunctions, includeEmptyStrings, serializedObj, propName, propValue) => { + serializedObj = {}; + if (!added.has(obj)) { + added.add(obj); + for (propName in obj) { + if (isValidMemberName(propName)) { + propValue = obj[propName]; + (includeFunctions || "function" != typeof propValue) && (includeEmptyStrings || "" !== propValue) && (serializedObj[propName] = serializeForWorker(winId, propValue, added)); + } + } + } + return serializedObj; + }; + const serializeCssRuleForWorker = cssRule => { + let obj = {}; + let key; + for (key in cssRule) { + validCssRuleProps.includes(key) && (obj[key] = String(cssRule[key])); + } + return obj; + }; + const deserializeFromWorker = (worker, serializedTransfer, serializedType, serializedValue) => { + if (serializedTransfer) { + serializedType = serializedTransfer[0]; + serializedValue = serializedTransfer[1]; + return 0 === serializedType ? serializedValue : 4 === serializedType ? deserializeRefFromWorker(worker, serializedValue) : 1 === serializedType ? serializedValue.map((v => deserializeFromWorker(worker, v))) : 3 === serializedType ? getInstance(serializedValue[0], serializedValue[1]) : 5 === serializedType ? constructEvent(deserializeObjectFromWorker(worker, serializedValue)) : 2 === serializedType ? deserializeObjectFromWorker(worker, serializedValue) : 8 === serializedType ? serializedValue : 9 === serializedType ? new window[serializedTransfer[2]](serializedValue) : void 0; + } + }; + const deserializeRefFromWorker = (worker, {$winId$: $winId$, $instanceId$: $instanceId$, $refId$: $refId$}, ref) => { + ref = mainRefs.get($refId$); + if (!ref) { + ref = function(...args) { + worker.postMessage([ 9, { + $winId$: $winId$, + $instanceId$: $instanceId$, + $refId$: $refId$, + $thisArg$: serializeForWorker($winId$, this), + $args$: serializeForWorker($winId$, args) + } ]); + }; + mainRefs.set($refId$, ref); + } + return ref; + }; + const constructEvent = eventProps => new ("detail" in eventProps ? CustomEvent : Event)(eventProps.type, eventProps); + const deserializeObjectFromWorker = (worker, serializedValue, obj, key) => { + obj = {}; + for (key in serializedValue) { + obj[key] = deserializeFromWorker(worker, serializedValue[key]); + } + return obj; + }; + const validCssRuleProps = "cssText,selectorText,href,media,namespaceURI,prefix,name,conditionText".split(","); + const mainAccessHandler = async (worker, accessReq) => { + let accessRsp = { + $msgId$: accessReq.$msgId$ + }; + let totalTasks = len(accessReq.$tasks$); + let i = 0; + let task; + let winId; + let applyPath; + let instance; + let rtnValue; + let isLast; + for (;i < totalTasks; i++) { + try { + isLast = i === totalTasks - 1; + task = accessReq.$tasks$[i]; + winId = task.$winId$; + applyPath = task.$applyPath$; + !winCtxs[winId] && winId.startsWith("f_") && await new Promise((resolve => { + let check = 0; + let callback = () => { + winCtxs[winId] || check++ > 1e3 ? resolve() : requestAnimationFrame(callback); + }; + callback(); + })); + if (1 === applyPath[0] && applyPath[1] in winCtxs[winId].$window$) { + setInstanceId(new winCtxs[winId].$window$[applyPath[1]](...deserializeFromWorker(worker, applyPath[2])), task.$instanceId$); + } else { + instance = getInstance(winId, task.$instanceId$); + if (instance) { + rtnValue = applyToInstance(worker, winId, instance, applyPath, isLast, task.$groupedGetters$); + task.$assignInstanceId$ && ("string" == typeof task.$assignInstanceId$ ? setInstanceId(rtnValue, task.$assignInstanceId$) : winCtxs[task.$assignInstanceId$.$winId$] = { + $winId$: task.$assignInstanceId$.$winId$, + $window$: { + document: rtnValue + } + }); + if (isPromise(rtnValue)) { + rtnValue = await rtnValue; + isLast && (accessRsp.$isPromise$ = true); + } + isLast && (accessRsp.$rtnValue$ = serializeForWorker(winId, rtnValue)); + } else { + accessRsp.$error$ = `Error finding instance "${task.$instanceId$}" on window ${normalizedWinId(winId)}`; + console.error(accessRsp.$error$, task); + } + } + } catch (e) { + isLast ? accessRsp.$error$ = String(e.stack || e) : console.error(e); + } + } + return accessRsp; + }; + const applyToInstance = (worker, winId, instance, applyPath, isLast, groupedGetters) => { + let i = 0; + let l = len(applyPath); + let next; + let current; + let previous; + let args; + let groupedRtnValues; + for (;i < l; i++) { + current = applyPath[i]; + next = applyPath[i + 1]; + previous = applyPath[i - 1]; + try { + if (!Array.isArray(next)) { + if ("string" == typeof current || "number" == typeof current) { + if (i + 1 === l && groupedGetters) { + groupedRtnValues = {}; + groupedGetters.map((propName => groupedRtnValues[propName] = instance[propName])); + return groupedRtnValues; + } + instance = instance[current]; + } else { + if (0 === next) { + instance[previous] = deserializeFromWorker(worker, current); + return; + } + if ("function" == typeof instance[previous]) { + args = deserializeFromWorker(worker, current); + "define" === previous && "CustomElementRegistry" === getConstructorName(instance) && (args[1] = defineCustomElement(winId, worker, args[1])); + "insertRule" === previous && args[1] > len(instance.cssRules) && (args[1] = len(instance.cssRules)); + instance = instance[previous].apply(instance, args); + if ("play" === previous) { + return Promise.resolve(); + } + } + } + } + } catch (err) { + if (isLast) { + throw err; + } + console.debug("Non-blocking setter error:", err); + } + } + return instance; + }; + const registerWindow = (worker, $winId$, $window$) => { + if (!windowIds.has($window$)) { + windowIds.set($window$, $winId$); + const doc = $window$.document; + const history = $window$.history; + const $parentWinId$ = windowIds.get($window$.parent); + const sendInitEnvData = () => worker.postMessage([ 5, { + $winId$: $winId$, + $parentWinId$: $parentWinId$, + $url$: doc.baseURI, + $visibilityState$: doc.visibilityState + } ]); + const pushState = history.pushState.bind(history); + const replaceState = history.replaceState.bind(history); + const onLocationChange = () => setTimeout((() => worker.postMessage([ 13, $winId$, doc.baseURI ]))); + history.pushState = (data, _, url) => { + pushState(data, _, url); + onLocationChange(); + }; + history.replaceState = (data, _, url) => { + replaceState(data, _, url); + onLocationChange(); + }; + $window$.addEventListener("popstate", onLocationChange); + $window$.addEventListener("hashchange", onLocationChange); + doc.addEventListener("visibilitychange", (() => worker.postMessage([ 14, $winId$, doc.visibilityState ]))); + winCtxs[$winId$] = { + $winId$: $winId$, + $window$: $window$ + }; + winCtxs[$winId$].$startTime$ = performance.now(); + { + const winType = $winId$ === $parentWinId$ ? "top" : "iframe"; + logMain(`Registered ${winType} window ${normalizedWinId($winId$)}`); + } + "complete" === doc.readyState ? sendInitEnvData() : $window$.addEventListener("load", sendInitEnvData); + } + }; + const readNextScript = (worker, winCtx) => { + let $winId$ = winCtx.$winId$; + let win = winCtx.$window$; + let doc = win.document; + let scriptSelector = 'script[type="text/partytown"]:not([data-ptid]):not([data-pterror])'; + let scriptElm; + let $instanceId$; + let scriptData; + if (doc && doc.body) { + scriptElm = doc.querySelector('script[type="text/partytown"]:not([data-ptid]):not([data-pterror]):not([async]):not([defer])'); + scriptElm || (scriptElm = doc.querySelector(scriptSelector)); + if (scriptElm) { + scriptElm.dataset.ptid = $instanceId$ = getAndSetInstanceId(scriptElm, $winId$); + scriptData = { + $winId$: $winId$, + $instanceId$: $instanceId$ + }; + if (scriptElm.src) { + scriptData.$url$ = scriptElm.src; + scriptData.$orgUrl$ = scriptElm.dataset.ptsrc || scriptElm.src; + } else { + scriptData.$content$ = scriptElm.innerHTML; + } + worker.postMessage([ 7, scriptData ]); + } else { + if (!winCtx.$isInitialized$) { + winCtx.$isInitialized$ = 1; + ((worker, $winId$, win) => { + let queuedForwardCalls = win._ptf; + let forwards = (win.partytown || {}).forward || []; + let i; + let mainForwardFn; + let forwardCall = ($forward$, args) => worker.postMessage([ 10, { + $winId$: $winId$, + $forward$: $forward$, + $args$: serializeForWorker($winId$, Array.from(args)) + } ]); + win._ptf = void 0; + forwards.map((forwardProps => { + mainForwardFn = win; + forwardProps.split(".").map(((_, i, arr) => { + mainForwardFn = mainForwardFn[arr[i]] = i + 1 < len(arr) ? mainForwardFn[arr[i]] || ("push" === arr[i + 1] ? [] : {}) : (...args) => forwardCall(arr, args); + })); + })); + if (queuedForwardCalls) { + for (i = 0; i < len(queuedForwardCalls); i += 2) { + forwardCall(queuedForwardCalls[i], queuedForwardCalls[i + 1]); + } + } + })(worker, $winId$, win); + doc.dispatchEvent(new CustomEvent("pt0")); + { + const winType = win === win.top ? "top" : "iframe"; + logMain(`Executed ${winType} window ${normalizedWinId($winId$)} environment scripts in ${(performance.now() - winCtx.$startTime$).toFixed(1)}ms`); + } + } + worker.postMessage([ 8, $winId$ ]); + } + } else { + requestAnimationFrame((() => readNextScript(worker, winCtx))); + } + }; + const onMessageFromWebWorker = (worker, msg, winCtx) => { + if (4 === msg[0]) { + registerWindow(worker, randomId(), mainWindow); + } else { + winCtx = winCtxs[msg[1]]; + winCtx && (7 === msg[0] ? requestAnimationFrame((() => readNextScript(worker, winCtx))) : 6 === msg[0] && ((worker, winCtx, instanceId, errorMsg, scriptElm) => { + scriptElm = winCtx.$window$.document.querySelector(`[data-ptid="${instanceId}"]`); + if (scriptElm) { + errorMsg ? scriptElm.dataset.pterror = errorMsg : scriptElm.type += "-x"; + delete scriptElm.dataset.ptid; + } + readNextScript(worker, winCtx); + })(worker, winCtx, msg[2], msg[3])); + } + }; + const readMainInterfaces = () => { + const elms = Object.getOwnPropertyNames(mainWindow).map((interfaceName => ((doc, interfaceName, r, tag) => { + r = interfaceName.match(/^(HTML|SVG)(.+)Element$/); + if (r) { + tag = r[2]; + return "S" == interfaceName[0] ? doc.createElementNS("http://www.w3.org/2000/svg", svgConstructorTags[tag] || tag.slice(0, 2).toLowerCase() + tag.slice(2)) : doc.createElement(htmlConstructorTags[tag] || tag); + } + })(docImpl, interfaceName))).filter((elm => elm)).map((elm => [ elm ])); + return readImplementations(elms, []); + }; + const cstrs = new Set([ "Object" ]); + const readImplementations = (impls, interfaces) => { + const cstrImpls = impls.filter((implData => implData[0])).map((implData => { + const impl = implData[0]; + const interfaceType = implData[1]; + const cstrName = getConstructorName(impl); + const CstrPrototype = mainWindow[cstrName].prototype; + return [ cstrName, CstrPrototype, impl, interfaceType ]; + })); + cstrImpls.map((([cstrName, CstrPrototype, impl, intefaceType]) => readOwnImplementation(cstrs, interfaces, cstrName, CstrPrototype, impl, intefaceType))); + return interfaces; + }; + const readImplementation = (cstrName, impl, memberName) => { + let interfaceMembers = []; + let interfaceInfo = [ cstrName, "Object", interfaceMembers ]; + for (memberName in impl) { + readImplementationMember(interfaceMembers, impl, memberName); + } + return interfaceInfo; + }; + const readOwnImplementation = (cstrs, interfaces, cstrName, CstrPrototype, impl, interfaceType) => { + if (!cstrs.has(cstrName)) { + cstrs.add(cstrName); + const SuperCstr = Object.getPrototypeOf(CstrPrototype); + const superCstrName = getConstructorName(SuperCstr); + const interfaceMembers = []; + const propDescriptors = Object.getOwnPropertyDescriptors(CstrPrototype); + readOwnImplementation(cstrs, interfaces, superCstrName, SuperCstr, impl, interfaceType); + for (const memberName in propDescriptors) { + readImplementationMember(interfaceMembers, impl, memberName); + } + interfaces.push([ cstrName, superCstrName, interfaceMembers, interfaceType, getNodeName(impl) ]); + } + }; + const readImplementationMember = (interfaceMembers, implementation, memberName, value, memberType, cstrName) => { + try { + if (isValidMemberName(memberName) && isNaN(memberName[0]) && "all" !== memberName) { + value = implementation[memberName]; + memberType = typeof value; + if ("function" === memberType) { + (String(value).includes("[native") || Object.getPrototypeOf(implementation)[memberName]) && interfaceMembers.push([ memberName, 5 ]); + } else if ("object" === memberType && null != value) { + cstrName = getConstructorName(value); + "Object" !== cstrName && self[cstrName] && interfaceMembers.push([ memberName, value.nodeType || cstrName ]); + } else { + "symbol" !== memberType && (memberName.toUpperCase() === memberName ? interfaceMembers.push([ memberName, 6, value ]) : interfaceMembers.push([ memberName, 6 ])); + } + } + } catch (e) { + console.warn(e); + } + }; + const readStorage = storageName => { + let items = []; + let i = 0; + let l = len(mainWindow[storageName]); + let key; + for (;i < l; i++) { + key = mainWindow[storageName].key(i); + items.push([ key, mainWindow[storageName].getItem(key) ]); + } + return items; + }; + const getGlobalConstructor = (mainWindow, cstrName) => void 0 !== mainWindow[cstrName] ? new mainWindow[cstrName](noop) : 0; + const addGlobalConstructorUsingPrototype = ($interfaces$, mainWindow, cstrName) => { + void 0 !== mainWindow[cstrName] && $interfaces$.push([ cstrName, "Object", Object.keys(mainWindow[cstrName].prototype).map((propName => [ propName, 6 ])), 12 ]); + }; + let worker; + (async receiveMessage => { + const sharedDataBuffer = new SharedArrayBuffer(1073741824); + const sharedData = new Int32Array(sharedDataBuffer); + return (worker, msg) => { + const msgType = msg[0]; + const accessReq = msg[1]; + if (0 === msgType) { + const initData = (() => { + const elm = docImpl.createElement("i"); + const textNode = docImpl.createTextNode(""); + const comment = docImpl.createComment(""); + const frag = docImpl.createDocumentFragment(); + const shadowRoot = docImpl.createElement("p").attachShadow({ + mode: "open" + }); + const intersectionObserver = getGlobalConstructor(mainWindow, "IntersectionObserver"); + const mutationObserver = getGlobalConstructor(mainWindow, "MutationObserver"); + const resizeObserver = getGlobalConstructor(mainWindow, "ResizeObserver"); + const perf = mainWindow.performance; + const screen = mainWindow.screen; + const impls = [ [ mainWindow.history ], [ perf ], [ perf.navigation ], [ perf.timing ], [ screen ], [ screen.orientation ], [ mainWindow.visualViewport ], [ intersectionObserver, 12 ], [ mutationObserver, 12 ], [ resizeObserver, 12 ], [ textNode ], [ comment ], [ frag ], [ shadowRoot ], [ elm ], [ elm.attributes ], [ elm.classList ], [ elm.dataset ], [ elm.style ], [ docImpl ], [ docImpl.doctype ] ]; + const initialInterfaces = [ readImplementation("Window", mainWindow), readImplementation("Node", textNode) ]; + const $config$ = JSON.stringify(config, ((k, v) => { + if ("function" == typeof v) { + v = String(v); + v.startsWith(k + "(") && (v = "function " + v); + } + return v; + })); + const initWebWorkerData = { + $config$: $config$, + $interfaces$: readImplementations(impls, initialInterfaces), + $libPath$: new URL(libPath, mainWindow.location) + "", + $origin$: origin, + $localStorage$: readStorage("localStorage"), + $sessionStorage$: readStorage("sessionStorage") + }; + addGlobalConstructorUsingPrototype(initWebWorkerData.$interfaces$, mainWindow, "IntersectionObserverEntry"); + return initWebWorkerData; + })(); + initData.$sharedDataBuffer$ = sharedDataBuffer; + worker.postMessage([ 1, initData ]); + } else { + 2 === msg[0] ? worker.postMessage([ 3, readMainInterfaces() ]) : 11 === msgType ? receiveMessage(accessReq, (accessRsp => { + const stringifiedData = JSON.stringify(accessRsp); + const stringifiedDataLength = stringifiedData.length; + for (let i = 0; i < stringifiedDataLength; i++) { + sharedData[i + 1] = stringifiedData.charCodeAt(i); + } + sharedData[0] = stringifiedDataLength; + Atomics.notify(sharedData, 0); + })) : onMessageFromWebWorker(worker, msg); + } + }; + })(((accessReq, responseCallback) => mainAccessHandler(worker, accessReq).then(responseCallback))).then((onMessageHandler => { + if (onMessageHandler) { + worker = new Worker(libPath + "partytown-ww-atomics.js?v=0.5.4", { + name: "Partytown 🎉" + }); + worker.onmessage = ev => { + const msg = ev.data; + 12 === msg[0] ? mainAccessHandler(worker, msg[1]) : onMessageHandler(worker, msg); + }; + logMain("Created Partytown web worker (0.5.4)"); + worker.onerror = ev => console.error("Web Worker Error", ev); + mainWindow.addEventListener("pt1", (ev => registerWindow(worker, getAndSetInstanceId(ev.detail.frameElement), ev.detail))); + } + })); +})(window); diff --git a/website/public/~partytown/debug/partytown-media.js b/website/public/~partytown/debug/partytown-media.js new file mode 100644 index 0000000000..bff2dd90e6 --- /dev/null +++ b/website/public/~partytown/debug/partytown-media.js @@ -0,0 +1,374 @@ +/* Partytown 0.5.4 - MIT builder.io */ +(self => { + const [getter, setter, callMethod, constructGlobal, definePrototypePropertyDescriptor, randomId, WinIdKey, InstanceIdKey, ApplyPathKey] = self.$bridgeToMedia$; + delete self.$bridgeToMedia$; + const ContextKey = Symbol(); + const MediaSourceKey = Symbol(); + const ReadyStateKey = Symbol(); + const SourceBuffersKey = Symbol(); + const SourceBufferTasksKey = Symbol(); + const TimeRangesKey = Symbol(); + const EMPTY_ARRAY = []; + const defineCstr = (win, cstrName, Cstr) => win[cstrName] = defineCstrName(cstrName, Cstr); + const defineCstrName = (cstrName, Cstr) => Object.defineProperty(Cstr, "name", { + value: cstrName + }); + const initCanvas = (WorkerBase, win) => { + const HTMLCanvasDescriptorMap = { + getContext: { + value(contextType, contextAttributes) { + this[ContextKey] || (this[ContextKey] = (contextType.includes("webgl") ? createContextWebGL : createContext2D)(this, contextType, contextAttributes)); + return this[ContextKey]; + } + } + }; + const WorkerCanvasGradient = defineCstr(win, "CanvasGradient", class extends WorkerBase { + addColorStop(...args) { + callMethod(this, [ "addColorStop" ], args, 2); + } + }); + const WorkerCanvasPattern = defineCstr(win, "CanvasPattern", class extends WorkerBase { + setTransform(...args) { + callMethod(this, [ "setTransform" ], args, 2); + } + }); + const createContext2D = (canvasInstance, contextType, contextAttributes) => { + const winId = canvasInstance[WinIdKey]; + const ctxInstanceId = randomId(); + const ctxInstance = { + [WinIdKey]: winId, + [InstanceIdKey]: ctxInstanceId, + [ApplyPathKey]: [] + }; + const ctx = callMethod(canvasInstance, [ "getContext" ], [ contextType, contextAttributes ], 1, ctxInstanceId); + const ctx2dGetterMethods = "getContextAttributes,getImageData,getLineDash,getTransform,isPointInPath,isPointInStroke,measureText".split(","); + const CanvasRenderingContext2D = { + get: (target, propName) => "string" == typeof propName && propName in ctx ? "function" == typeof ctx[propName] ? (...args) => { + if (propName.startsWith("create")) { + const instanceId = randomId(); + callMethod(ctxInstance, [ propName ], args, 2, instanceId); + if ("createImageData" === propName || "createPattern" === propName) { + (api => { + console.warn(`${api} not implemented`); + })(`${propName}()`); + return { + setTransform: () => {} + }; + } + return new WorkerCanvasGradient(winId, instanceId); + } + const methodCallType = ctx2dGetterMethods.includes(propName) ? 1 : 2; + return callMethod(ctxInstance, [ propName ], args, methodCallType); + } : ctx[propName] : target[propName], + set(target, propName, value) { + if ("string" == typeof propName && propName in ctx) { + ctx[propName] !== value && "function" != typeof value && setter(ctxInstance, [ propName ], value); + ctx[propName] = value; + } else { + target[propName] = value; + } + return true; + } + }; + return new Proxy(ctx, CanvasRenderingContext2D); + }; + const createContextWebGL = (canvasInstance, contextType, contextAttributes) => { + const winId = canvasInstance[WinIdKey]; + const ctxInstanceId = randomId(); + const ctxInstance = { + [WinIdKey]: winId, + [InstanceIdKey]: ctxInstanceId, + [ApplyPathKey]: [] + }; + const ctx = callMethod(canvasInstance, [ "getContext" ], [ contextType, contextAttributes ], 1, ctxInstanceId); + const WebGLRenderingContextHandler = { + get: (target, propName) => "string" == typeof propName ? "function" != typeof ctx[propName] ? ctx[propName] : (...args) => callMethod(ctxInstance, [ propName ], args, getWebGlMethodCallType(propName)) : target[propName], + set(target, propName, value) { + if ("string" == typeof propName && propName in ctx) { + ctx[propName] !== value && "function" != typeof value && setter(ctxInstance, [ propName ], value); + ctx[propName] = value; + } else { + target[propName] = value; + } + return true; + } + }; + return new Proxy(ctx, WebGLRenderingContextHandler); + }; + const ctxWebGLGetterMethods = "checkFramebufferStatus,makeXRCompatible".split(","); + const getWebGlMethodCallType = methodName => methodName.startsWith("create") || methodName.startsWith("get") || methodName.startsWith("is") || ctxWebGLGetterMethods.includes(methodName) ? 1 : 2; + defineCstr(win, "CanvasGradient", WorkerCanvasGradient); + defineCstr(win, "CanvasPattern", WorkerCanvasPattern); + definePrototypePropertyDescriptor(win.HTMLCanvasElement, HTMLCanvasDescriptorMap); + }; + const initMedia = (WorkerBase, WorkerEventTargetProxy, env, win) => { + var _a, _b; + win.Audio = defineCstrName("HTMLAudioElement", class { + constructor(src) { + const audio = env.$createNode$("audio", randomId()); + audio.src = src; + return audio; + } + }); + const WorkerAudioTrack = class extends WorkerBase { + get enabled() { + return getter(this, [ "enabled" ]); + } + set enabled(value) { + setter(this, [ "enabled" ], value); + } + get id() { + return getter(this, [ "id" ]); + } + get kind() { + return getter(this, [ "kind" ]); + } + get label() { + return getter(this, [ "label" ]); + } + get language() { + return getter(this, [ "language" ]); + } + get sourceBuffer() { + return new WorkerSourceBuffer(this); + } + }; + const WorkerAudioTrackList = class { + constructor(mediaElm) { + const winId = mediaElm[WinIdKey]; + const instanceId = mediaElm[InstanceIdKey]; + const instance = { + addEventListener(...args) { + callMethod(mediaElm, [ "audioTracks", "addEventListener" ], args, 3); + }, + getTrackById: (...args) => callMethod(mediaElm, [ "audioTracks", "getTrackById" ], args), + get length() { + return getter(mediaElm, [ "audioTracks", "length" ]); + }, + removeEventListener(...args) { + callMethod(mediaElm, [ "audioTracks", "removeEventListener" ], args, 3); + } + }; + return new Proxy(instance, { + get: (target, propName) => "number" == typeof propName ? new WorkerAudioTrack(winId, instanceId, [ "audioTracks", propName ]) : target[propName] + }); + } + }; + const WorkerSourceBufferList = defineCstr(win, "SourceBufferList", class extends Array { + constructor(mediaSource) { + super(); + this[MediaSourceKey] = mediaSource; + } + addEventListener(...args) { + callMethod(this[MediaSourceKey], [ "sourceBuffers", "addEventListener" ], args, 3); + } + removeEventListener(...args) { + callMethod(this[MediaSourceKey], [ "sourceBuffers", "removeEventListener" ], args, 3); + } + }); + const WorkerSourceBuffer = defineCstr(win, "SourceBuffer", (_b = class extends WorkerEventTargetProxy { + constructor(mediaSource) { + super(mediaSource[WinIdKey], mediaSource[InstanceIdKey], [ "sourceBuffers" ]); + this[_a] = []; + this[MediaSourceKey] = mediaSource; + } + abort() { + const sbIndex = getSourceBufferIndex(this); + callMethod(this, [ sbIndex, "appendWindowStart" ], EMPTY_ARRAY, 1); + } + addEventListener(...args) { + const sbIndex = getSourceBufferIndex(this); + callMethod(this, [ sbIndex, "addEventListener" ], args, 3); + } + appendBuffer(buf) { + this[SourceBufferTasksKey].push([ "appendBuffer", [ buf ], buf ]); + drainSourceBufferQueue(this); + } + get appendWindowStart() { + const sbIndex = getSourceBufferIndex(this); + return getter(this, [ sbIndex, "appendWindowStart" ]); + } + set appendWindowStart(value) { + const sbIndex = getSourceBufferIndex(this); + setter(this, [ sbIndex, "appendWindowStart" ], value); + } + get appendWindowEnd() { + const sbIndex = getSourceBufferIndex(this); + return getter(this, [ sbIndex, "appendWindowEnd" ]); + } + set appendWindowEnd(value) { + const sbIndex = getSourceBufferIndex(this); + setter(this, [ sbIndex, "appendWindowEnd" ], value); + } + get buffered() { + const mediaSource = this[MediaSourceKey]; + const sbIndex = getSourceBufferIndex(this); + const timeRanges = new WorkerTimeRanges(mediaSource[WinIdKey], mediaSource[InstanceIdKey], [ "sourceBuffers", sbIndex, "buffered" ]); + return timeRanges; + } + changeType(mimeType) { + const sbIndex = getSourceBufferIndex(this); + callMethod(this, [ sbIndex, "changeType" ], [ mimeType ], 2); + } + get mode() { + const sbIndex = getSourceBufferIndex(this); + return getter(this, [ sbIndex, "mode" ]); + } + set mode(value) { + const sbIndex = getSourceBufferIndex(this); + setter(this, [ sbIndex, "mode" ], value); + } + remove(start, end) { + this[SourceBufferTasksKey].push([ "remove", [ start, end ] ]); + drainSourceBufferQueue(this); + } + removeEventListener(...args) { + const sbIndex = getSourceBufferIndex(this); + callMethod(this, [ sbIndex, "removeEventListener" ], args, 3); + } + get timestampOffset() { + const sbIndex = getSourceBufferIndex(this); + return getter(this, [ sbIndex, "timestampOffset" ]); + } + set timestampOffset(value) { + const sbIndex = getSourceBufferIndex(this); + setter(this, [ sbIndex, "timestampOffset" ], value); + } + get updating() { + const sbIndex = getSourceBufferIndex(this); + return getter(this, [ sbIndex, "updating" ]); + } + }, _a = SourceBufferTasksKey, _b)); + const WorkerTimeRanges = defineCstr(win, "TimeRanges", class extends WorkerBase { + start(...args) { + return callMethod(this, [ "start" ], args); + } + end(...args) { + return callMethod(this, [ "end" ], args); + } + get length() { + return getter(this, [ "length" ]); + } + }); + const getSourceBufferIndex = sourceBuffer => { + if (sourceBuffer) { + const mediaSource = sourceBuffer[MediaSourceKey]; + const sourceBufferList = mediaSource[SourceBuffersKey]; + return sourceBufferList.indexOf(sourceBuffer); + } + return -1; + }; + const drainSourceBufferQueue = sourceBuffer => { + if (sourceBuffer[SourceBufferTasksKey].length) { + if (!sourceBuffer.updating) { + const task = sourceBuffer[SourceBufferTasksKey].shift(); + if (task) { + const sbIndex = getSourceBufferIndex(sourceBuffer); + callMethod(sourceBuffer, [ sbIndex, task[0] ], task[1], 3, void 0, task[2]); + } + } + setTimeout((() => drainSourceBufferQueue(sourceBuffer)), 50); + } + }; + const HTMLMediaDescriptorMap = { + buffered: { + get() { + if (!this[TimeRangesKey]) { + this[TimeRangesKey] = new WorkerTimeRanges(this[WinIdKey], this[InstanceIdKey], [ "buffered" ]); + setTimeout((() => { + this[TimeRangesKey] = void 0; + }), 5e3); + } + return this[TimeRangesKey]; + } + }, + readyState: { + get() { + if (4 === this[ReadyStateKey]) { + return 4; + } + if ("number" != typeof this[ReadyStateKey]) { + this[ReadyStateKey] = getter(this, [ "readyState" ]); + setTimeout((() => { + this[ReadyStateKey] = void 0; + }), 1e3); + } + return this[ReadyStateKey]; + } + } + }; + defineCstr(win, "MediaSource", class extends WorkerEventTargetProxy { + constructor() { + super(env.$winId$); + this[SourceBuffersKey] = new WorkerSourceBufferList(this); + constructGlobal(this, "MediaSource", EMPTY_ARRAY); + } + get activeSourceBuffers() { + return []; + } + addSourceBuffer(mimeType) { + const sourceBuffer = new WorkerSourceBuffer(this); + this[SourceBuffersKey].push(sourceBuffer); + callMethod(this, [ "addSourceBuffer" ], [ mimeType ]); + return sourceBuffer; + } + clearLiveSeekableRange() { + callMethod(this, [ "clearLiveSeekableRange" ], EMPTY_ARRAY, 2); + } + get duration() { + return getter(this, [ "duration" ]); + } + set duration(value) { + setter(this, [ "duration" ], value); + } + endOfStream(endOfStreamError) { + callMethod(this, [ "endOfStream" ], [ endOfStreamError ], 3); + } + get readyState() { + return getter(this, [ "readyState" ]); + } + removeSourceBuffer(sourceBuffer) { + const index = getSourceBufferIndex(sourceBuffer); + if (index > -1) { + this[SourceBuffersKey].splice(index, 1); + callMethod(this, [ "removeSourceBuffer" ], [ index ], 1); + } + } + setLiveSeekableRange(start, end) { + callMethod(this, [ "setLiveSeekableRange" ], [ start, end ], 2); + } + get sourceBuffers() { + return this[SourceBuffersKey]; + } + static isTypeSupported(mimeType) { + if (!isStaticTypeSupported.has(mimeType)) { + const isSupported = callMethod(win, [ "MediaSource", "isTypeSupported" ], [ mimeType ]); + isStaticTypeSupported.set(mimeType, isSupported); + } + return isStaticTypeSupported.get(mimeType); + } + }); + const winURL = win.URL = defineCstrName("URL", class extends URL {}); + const hasAudioTracks = "audioTracks" in win.HTMLMediaElement.prototype; + if (hasAudioTracks) { + defineCstr(win, "AudioTrackList", WorkerAudioTrackList); + defineCstr(win, "AudioTrack", WorkerAudioTrack); + HTMLMediaDescriptorMap.audioTracks = { + get() { + return new WorkerAudioTrackList(this); + } + }; + } + definePrototypePropertyDescriptor(win.HTMLMediaElement, HTMLMediaDescriptorMap); + winURL.createObjectURL = obj => callMethod(win, [ "URL", "createObjectURL" ], [ obj ]); + winURL.revokeObjectURL = obj => callMethod(win, [ "URL", "revokeObjectURL" ], [ obj ]); + }; + const isStaticTypeSupported = new Map; + self.$bridgeFromMedia$ = (WorkerBase, WorkerEventTargetProxy, env, win, windowMediaConstructors) => { + windowMediaConstructors.map((mediaCstrName => { + delete win[mediaCstrName]; + })); + initCanvas(WorkerBase, win); + initMedia(WorkerBase, WorkerEventTargetProxy, env, win); + }; +})(self); diff --git a/website/public/~partytown/debug/partytown-sandbox-sw.js b/website/public/~partytown/debug/partytown-sandbox-sw.js new file mode 100644 index 0000000000..9ca9e5f127 --- /dev/null +++ b/website/public/~partytown/debug/partytown-sandbox-sw.js @@ -0,0 +1,523 @@ +/* Partytown 0.5.4 - MIT builder.io */ +(window => { + const isPromise = v => "object" == typeof v && v && v.then; + const noop = () => {}; + const len = obj => obj.length; + const getConstructorName = obj => { + var _a, _b, _c; + try { + const constructorName = null === (_a = null == obj ? void 0 : obj.constructor) || void 0 === _a ? void 0 : _a.name; + if (constructorName) { + return constructorName; + } + } catch (e) {} + try { + const zoneJsConstructorName = null === (_c = null === (_b = null == obj ? void 0 : obj.__zone_symbol__originalInstance) || void 0 === _b ? void 0 : _b.constructor) || void 0 === _c ? void 0 : _c.name; + if (zoneJsConstructorName) { + return zoneJsConstructorName; + } + } catch (e) {} + return ""; + }; + const startsWith = (str, val) => str.startsWith(val); + const isValidMemberName = memberName => !(startsWith(memberName, "webkit") || startsWith(memberName, "toJSON") || startsWith(memberName, "constructor") || startsWith(memberName, "toString") || startsWith(memberName, "_")); + const getNodeName = node => 11 === node.nodeType && node.host ? "#s" : node.nodeName; + const randomId = () => Math.round(Math.random() * Number.MAX_SAFE_INTEGER).toString(36); + const defineConstructorName = (Cstr, value) => ((obj, memberName, descriptor) => Object.defineProperty(obj, memberName, { + ...descriptor, + configurable: true + }))(Cstr, "name", { + value: value + }); + const htmlConstructorTags = { + Anchor: "a", + DList: "dl", + Image: "img", + OList: "ol", + Paragraph: "p", + Quote: "q", + TableCaption: "caption", + TableCell: "td", + TableCol: "colgroup", + TableRow: "tr", + TableSection: "tbody", + UList: "ul" + }; + const svgConstructorTags = { + Graphics: "g", + SVG: "svg" + }; + const InstanceIdKey = Symbol(); + const CreatedKey = Symbol(); + const instances = new Map; + const mainRefs = new Map; + const winCtxs = {}; + const windowIds = new WeakMap; + const getAndSetInstanceId = (instance, instanceId) => { + if (instance) { + if (instanceId = windowIds.get(instance)) { + return instanceId; + } + (instanceId = instance[InstanceIdKey]) || setInstanceId(instance, instanceId = randomId()); + return instanceId; + } + }; + const getInstance = (winId, instanceId, win, doc, docId) => { + if ((win = winCtxs[winId]) && win.$window$) { + if (winId === instanceId) { + return win.$window$; + } + doc = win.$window$.document; + docId = instanceId.split(".").pop(); + if ("d" === docId) { + return doc; + } + if ("e" === docId) { + return doc.documentElement; + } + if ("h" === docId) { + return doc.head; + } + if ("b" === docId) { + return doc.body; + } + } + return instances.get(instanceId); + }; + const setInstanceId = (instance, instanceId, now) => { + if (instance) { + instances.set(instanceId, instance); + instance[InstanceIdKey] = instanceId; + instance[CreatedKey] = now = Date.now(); + if (now > lastCleanup + 5e3) { + instances.forEach(((storedInstance, instanceId) => { + storedInstance[CreatedKey] < lastCleanup && storedInstance.nodeType && !storedInstance.isConnected && instances.delete(instanceId); + })); + lastCleanup = now; + } + } + }; + let lastCleanup = 0; + const mainWindow = window.parent; + const docImpl = document.implementation.createHTMLDocument(); + const config = mainWindow.partytown || {}; + const libPath = (config.lib || "/~partytown/") + "debug/"; + const logMain = msg => { + console.debug.apply(console, [ "%cMain 🌎", "background: #717171; color: white; padding: 2px 3px; border-radius: 2px; font-size: 0.8em;", msg ]); + }; + const winIds = []; + const normalizedWinId = winId => { + winIds.includes(winId) || winIds.push(winId); + return winIds.indexOf(winId) + 1; + }; + const defineCustomElement = (winId, worker, ceData) => { + const Cstr = defineConstructorName(class extends winCtxs[winId].$window$.HTMLElement {}, ceData[0]); + const ceCallbackMethods = "connectedCallback,disconnectedCallback,attributeChangedCallback,adoptedCallback".split(","); + ceCallbackMethods.map((callbackMethodName => Cstr.prototype[callbackMethodName] = function(...args) { + worker.postMessage([ 15, winId, getAndSetInstanceId(this), callbackMethodName, args ]); + })); + Cstr.observedAttributes = ceData[1]; + return Cstr; + }; + const serializeForWorker = ($winId$, value, added, type, cstrName) => void 0 !== value && (type = typeof value) ? "string" === type || "number" === type || "boolean" === type || null == value ? [ 0, value ] : "function" === type ? [ 6 ] : (added = added || new Set) && Array.isArray(value) ? added.has(value) ? [ 1, [] ] : added.add(value) && [ 1, value.map((v => serializeForWorker($winId$, v, added))) ] : "object" === type ? "" === (cstrName = getConstructorName(value)) ? [ 2, {} ] : "Window" === cstrName ? [ 3, [ $winId$, $winId$ ] ] : "HTMLCollection" === cstrName || "NodeList" === cstrName ? [ 7, Array.from(value).map((v => serializeForWorker($winId$, v, added)[1])) ] : cstrName.endsWith("Event") ? [ 5, serializeObjectForWorker($winId$, value, added) ] : "CSSRuleList" === cstrName ? [ 12, Array.from(value).map(serializeCssRuleForWorker) ] : startsWith(cstrName, "CSS") && cstrName.endsWith("Rule") ? [ 11, serializeCssRuleForWorker(value) ] : "CSSStyleDeclaration" === cstrName ? [ 13, serializeObjectForWorker($winId$, value, added) ] : "Attr" === cstrName ? [ 10, [ value.name, value.value ] ] : value.nodeType ? [ 3, [ $winId$, getAndSetInstanceId(value), getNodeName(value) ] ] : [ 2, serializeObjectForWorker($winId$, value, added, true, true) ] : void 0 : value; + const serializeObjectForWorker = (winId, obj, added, includeFunctions, includeEmptyStrings, serializedObj, propName, propValue) => { + serializedObj = {}; + if (!added.has(obj)) { + added.add(obj); + for (propName in obj) { + if (isValidMemberName(propName)) { + propValue = obj[propName]; + (includeFunctions || "function" != typeof propValue) && (includeEmptyStrings || "" !== propValue) && (serializedObj[propName] = serializeForWorker(winId, propValue, added)); + } + } + } + return serializedObj; + }; + const serializeCssRuleForWorker = cssRule => { + let obj = {}; + let key; + for (key in cssRule) { + validCssRuleProps.includes(key) && (obj[key] = String(cssRule[key])); + } + return obj; + }; + const deserializeFromWorker = (worker, serializedTransfer, serializedType, serializedValue) => { + if (serializedTransfer) { + serializedType = serializedTransfer[0]; + serializedValue = serializedTransfer[1]; + return 0 === serializedType ? serializedValue : 4 === serializedType ? deserializeRefFromWorker(worker, serializedValue) : 1 === serializedType ? serializedValue.map((v => deserializeFromWorker(worker, v))) : 3 === serializedType ? getInstance(serializedValue[0], serializedValue[1]) : 5 === serializedType ? constructEvent(deserializeObjectFromWorker(worker, serializedValue)) : 2 === serializedType ? deserializeObjectFromWorker(worker, serializedValue) : 8 === serializedType ? serializedValue : 9 === serializedType ? new window[serializedTransfer[2]](serializedValue) : void 0; + } + }; + const deserializeRefFromWorker = (worker, {$winId$: $winId$, $instanceId$: $instanceId$, $refId$: $refId$}, ref) => { + ref = mainRefs.get($refId$); + if (!ref) { + ref = function(...args) { + worker.postMessage([ 9, { + $winId$: $winId$, + $instanceId$: $instanceId$, + $refId$: $refId$, + $thisArg$: serializeForWorker($winId$, this), + $args$: serializeForWorker($winId$, args) + } ]); + }; + mainRefs.set($refId$, ref); + } + return ref; + }; + const constructEvent = eventProps => new ("detail" in eventProps ? CustomEvent : Event)(eventProps.type, eventProps); + const deserializeObjectFromWorker = (worker, serializedValue, obj, key) => { + obj = {}; + for (key in serializedValue) { + obj[key] = deserializeFromWorker(worker, serializedValue[key]); + } + return obj; + }; + const validCssRuleProps = "cssText,selectorText,href,media,namespaceURI,prefix,name,conditionText".split(","); + const mainAccessHandler = async (worker, accessReq) => { + let accessRsp = { + $msgId$: accessReq.$msgId$ + }; + let totalTasks = len(accessReq.$tasks$); + let i = 0; + let task; + let winId; + let applyPath; + let instance; + let rtnValue; + let isLast; + for (;i < totalTasks; i++) { + try { + isLast = i === totalTasks - 1; + task = accessReq.$tasks$[i]; + winId = task.$winId$; + applyPath = task.$applyPath$; + !winCtxs[winId] && winId.startsWith("f_") && await new Promise((resolve => { + let check = 0; + let callback = () => { + winCtxs[winId] || check++ > 1e3 ? resolve() : requestAnimationFrame(callback); + }; + callback(); + })); + if (1 === applyPath[0] && applyPath[1] in winCtxs[winId].$window$) { + setInstanceId(new winCtxs[winId].$window$[applyPath[1]](...deserializeFromWorker(worker, applyPath[2])), task.$instanceId$); + } else { + instance = getInstance(winId, task.$instanceId$); + if (instance) { + rtnValue = applyToInstance(worker, winId, instance, applyPath, isLast, task.$groupedGetters$); + task.$assignInstanceId$ && ("string" == typeof task.$assignInstanceId$ ? setInstanceId(rtnValue, task.$assignInstanceId$) : winCtxs[task.$assignInstanceId$.$winId$] = { + $winId$: task.$assignInstanceId$.$winId$, + $window$: { + document: rtnValue + } + }); + if (isPromise(rtnValue)) { + rtnValue = await rtnValue; + isLast && (accessRsp.$isPromise$ = true); + } + isLast && (accessRsp.$rtnValue$ = serializeForWorker(winId, rtnValue)); + } else { + accessRsp.$error$ = `Error finding instance "${task.$instanceId$}" on window ${normalizedWinId(winId)}`; + console.error(accessRsp.$error$, task); + } + } + } catch (e) { + isLast ? accessRsp.$error$ = String(e.stack || e) : console.error(e); + } + } + return accessRsp; + }; + const applyToInstance = (worker, winId, instance, applyPath, isLast, groupedGetters) => { + let i = 0; + let l = len(applyPath); + let next; + let current; + let previous; + let args; + let groupedRtnValues; + for (;i < l; i++) { + current = applyPath[i]; + next = applyPath[i + 1]; + previous = applyPath[i - 1]; + try { + if (!Array.isArray(next)) { + if ("string" == typeof current || "number" == typeof current) { + if (i + 1 === l && groupedGetters) { + groupedRtnValues = {}; + groupedGetters.map((propName => groupedRtnValues[propName] = instance[propName])); + return groupedRtnValues; + } + instance = instance[current]; + } else { + if (0 === next) { + instance[previous] = deserializeFromWorker(worker, current); + return; + } + if ("function" == typeof instance[previous]) { + args = deserializeFromWorker(worker, current); + "define" === previous && "CustomElementRegistry" === getConstructorName(instance) && (args[1] = defineCustomElement(winId, worker, args[1])); + "insertRule" === previous && args[1] > len(instance.cssRules) && (args[1] = len(instance.cssRules)); + instance = instance[previous].apply(instance, args); + if ("play" === previous) { + return Promise.resolve(); + } + } + } + } + } catch (err) { + if (isLast) { + throw err; + } + console.debug("Non-blocking setter error:", err); + } + } + return instance; + }; + const registerWindow = (worker, $winId$, $window$) => { + if (!windowIds.has($window$)) { + windowIds.set($window$, $winId$); + const doc = $window$.document; + const history = $window$.history; + const $parentWinId$ = windowIds.get($window$.parent); + const sendInitEnvData = () => worker.postMessage([ 5, { + $winId$: $winId$, + $parentWinId$: $parentWinId$, + $url$: doc.baseURI, + $visibilityState$: doc.visibilityState + } ]); + const pushState = history.pushState.bind(history); + const replaceState = history.replaceState.bind(history); + const onLocationChange = () => setTimeout((() => worker.postMessage([ 13, $winId$, doc.baseURI ]))); + history.pushState = (data, _, url) => { + pushState(data, _, url); + onLocationChange(); + }; + history.replaceState = (data, _, url) => { + replaceState(data, _, url); + onLocationChange(); + }; + $window$.addEventListener("popstate", onLocationChange); + $window$.addEventListener("hashchange", onLocationChange); + doc.addEventListener("visibilitychange", (() => worker.postMessage([ 14, $winId$, doc.visibilityState ]))); + winCtxs[$winId$] = { + $winId$: $winId$, + $window$: $window$ + }; + winCtxs[$winId$].$startTime$ = performance.now(); + { + const winType = $winId$ === $parentWinId$ ? "top" : "iframe"; + logMain(`Registered ${winType} window ${normalizedWinId($winId$)}`); + } + "complete" === doc.readyState ? sendInitEnvData() : $window$.addEventListener("load", sendInitEnvData); + } + }; + const readNextScript = (worker, winCtx) => { + let $winId$ = winCtx.$winId$; + let win = winCtx.$window$; + let doc = win.document; + let scriptSelector = 'script[type="text/partytown"]:not([data-ptid]):not([data-pterror])'; + let scriptElm; + let $instanceId$; + let scriptData; + if (doc && doc.body) { + scriptElm = doc.querySelector('script[type="text/partytown"]:not([data-ptid]):not([data-pterror]):not([async]):not([defer])'); + scriptElm || (scriptElm = doc.querySelector(scriptSelector)); + if (scriptElm) { + scriptElm.dataset.ptid = $instanceId$ = getAndSetInstanceId(scriptElm, $winId$); + scriptData = { + $winId$: $winId$, + $instanceId$: $instanceId$ + }; + if (scriptElm.src) { + scriptData.$url$ = scriptElm.src; + scriptData.$orgUrl$ = scriptElm.dataset.ptsrc || scriptElm.src; + } else { + scriptData.$content$ = scriptElm.innerHTML; + } + worker.postMessage([ 7, scriptData ]); + } else { + if (!winCtx.$isInitialized$) { + winCtx.$isInitialized$ = 1; + ((worker, $winId$, win) => { + let queuedForwardCalls = win._ptf; + let forwards = (win.partytown || {}).forward || []; + let i; + let mainForwardFn; + let forwardCall = ($forward$, args) => worker.postMessage([ 10, { + $winId$: $winId$, + $forward$: $forward$, + $args$: serializeForWorker($winId$, Array.from(args)) + } ]); + win._ptf = void 0; + forwards.map((forwardProps => { + mainForwardFn = win; + forwardProps.split(".").map(((_, i, arr) => { + mainForwardFn = mainForwardFn[arr[i]] = i + 1 < len(arr) ? mainForwardFn[arr[i]] || ("push" === arr[i + 1] ? [] : {}) : (...args) => forwardCall(arr, args); + })); + })); + if (queuedForwardCalls) { + for (i = 0; i < len(queuedForwardCalls); i += 2) { + forwardCall(queuedForwardCalls[i], queuedForwardCalls[i + 1]); + } + } + })(worker, $winId$, win); + doc.dispatchEvent(new CustomEvent("pt0")); + { + const winType = win === win.top ? "top" : "iframe"; + logMain(`Executed ${winType} window ${normalizedWinId($winId$)} environment scripts in ${(performance.now() - winCtx.$startTime$).toFixed(1)}ms`); + } + } + worker.postMessage([ 8, $winId$ ]); + } + } else { + requestAnimationFrame((() => readNextScript(worker, winCtx))); + } + }; + const onMessageFromWebWorker = (worker, msg, winCtx) => { + if (4 === msg[0]) { + registerWindow(worker, randomId(), mainWindow); + } else { + winCtx = winCtxs[msg[1]]; + winCtx && (7 === msg[0] ? requestAnimationFrame((() => readNextScript(worker, winCtx))) : 6 === msg[0] && ((worker, winCtx, instanceId, errorMsg, scriptElm) => { + scriptElm = winCtx.$window$.document.querySelector(`[data-ptid="${instanceId}"]`); + if (scriptElm) { + errorMsg ? scriptElm.dataset.pterror = errorMsg : scriptElm.type += "-x"; + delete scriptElm.dataset.ptid; + } + readNextScript(worker, winCtx); + })(worker, winCtx, msg[2], msg[3])); + } + }; + const readMainPlatform = () => { + const elm = docImpl.createElement("i"); + const textNode = docImpl.createTextNode(""); + const comment = docImpl.createComment(""); + const frag = docImpl.createDocumentFragment(); + const shadowRoot = docImpl.createElement("p").attachShadow({ + mode: "open" + }); + const intersectionObserver = getGlobalConstructor(mainWindow, "IntersectionObserver"); + const mutationObserver = getGlobalConstructor(mainWindow, "MutationObserver"); + const resizeObserver = getGlobalConstructor(mainWindow, "ResizeObserver"); + const perf = mainWindow.performance; + const screen = mainWindow.screen; + const impls = [ [ mainWindow.history ], [ perf ], [ perf.navigation ], [ perf.timing ], [ screen ], [ screen.orientation ], [ mainWindow.visualViewport ], [ intersectionObserver, 12 ], [ mutationObserver, 12 ], [ resizeObserver, 12 ], [ textNode ], [ comment ], [ frag ], [ shadowRoot ], [ elm ], [ elm.attributes ], [ elm.classList ], [ elm.dataset ], [ elm.style ], [ docImpl ], [ docImpl.doctype ] ]; + const initialInterfaces = [ readImplementation("Window", mainWindow), readImplementation("Node", textNode) ]; + const $config$ = JSON.stringify(config, ((k, v) => { + if ("function" == typeof v) { + v = String(v); + v.startsWith(k + "(") && (v = "function " + v); + } + return v; + })); + const initWebWorkerData = { + $config$: $config$, + $interfaces$: readImplementations(impls, initialInterfaces), + $libPath$: new URL(libPath, mainWindow.location) + "", + $origin$: origin, + $localStorage$: readStorage("localStorage"), + $sessionStorage$: readStorage("sessionStorage") + }; + addGlobalConstructorUsingPrototype(initWebWorkerData.$interfaces$, mainWindow, "IntersectionObserverEntry"); + return initWebWorkerData; + }; + const readMainInterfaces = () => { + const elms = Object.getOwnPropertyNames(mainWindow).map((interfaceName => ((doc, interfaceName, r, tag) => { + r = interfaceName.match(/^(HTML|SVG)(.+)Element$/); + if (r) { + tag = r[2]; + return "S" == interfaceName[0] ? doc.createElementNS("http://www.w3.org/2000/svg", svgConstructorTags[tag] || tag.slice(0, 2).toLowerCase() + tag.slice(2)) : doc.createElement(htmlConstructorTags[tag] || tag); + } + })(docImpl, interfaceName))).filter((elm => elm)).map((elm => [ elm ])); + return readImplementations(elms, []); + }; + const cstrs = new Set([ "Object" ]); + const readImplementations = (impls, interfaces) => { + const cstrImpls = impls.filter((implData => implData[0])).map((implData => { + const impl = implData[0]; + const interfaceType = implData[1]; + const cstrName = getConstructorName(impl); + const CstrPrototype = mainWindow[cstrName].prototype; + return [ cstrName, CstrPrototype, impl, interfaceType ]; + })); + cstrImpls.map((([cstrName, CstrPrototype, impl, intefaceType]) => readOwnImplementation(cstrs, interfaces, cstrName, CstrPrototype, impl, intefaceType))); + return interfaces; + }; + const readImplementation = (cstrName, impl, memberName) => { + let interfaceMembers = []; + let interfaceInfo = [ cstrName, "Object", interfaceMembers ]; + for (memberName in impl) { + readImplementationMember(interfaceMembers, impl, memberName); + } + return interfaceInfo; + }; + const readOwnImplementation = (cstrs, interfaces, cstrName, CstrPrototype, impl, interfaceType) => { + if (!cstrs.has(cstrName)) { + cstrs.add(cstrName); + const SuperCstr = Object.getPrototypeOf(CstrPrototype); + const superCstrName = getConstructorName(SuperCstr); + const interfaceMembers = []; + const propDescriptors = Object.getOwnPropertyDescriptors(CstrPrototype); + readOwnImplementation(cstrs, interfaces, superCstrName, SuperCstr, impl, interfaceType); + for (const memberName in propDescriptors) { + readImplementationMember(interfaceMembers, impl, memberName); + } + interfaces.push([ cstrName, superCstrName, interfaceMembers, interfaceType, getNodeName(impl) ]); + } + }; + const readImplementationMember = (interfaceMembers, implementation, memberName, value, memberType, cstrName) => { + try { + if (isValidMemberName(memberName) && isNaN(memberName[0]) && "all" !== memberName) { + value = implementation[memberName]; + memberType = typeof value; + if ("function" === memberType) { + (String(value).includes("[native") || Object.getPrototypeOf(implementation)[memberName]) && interfaceMembers.push([ memberName, 5 ]); + } else if ("object" === memberType && null != value) { + cstrName = getConstructorName(value); + "Object" !== cstrName && self[cstrName] && interfaceMembers.push([ memberName, value.nodeType || cstrName ]); + } else { + "symbol" !== memberType && (memberName.toUpperCase() === memberName ? interfaceMembers.push([ memberName, 6, value ]) : interfaceMembers.push([ memberName, 6 ])); + } + } + } catch (e) { + console.warn(e); + } + }; + const readStorage = storageName => { + let items = []; + let i = 0; + let l = len(mainWindow[storageName]); + let key; + for (;i < l; i++) { + key = mainWindow[storageName].key(i); + items.push([ key, mainWindow[storageName].getItem(key) ]); + } + return items; + }; + const getGlobalConstructor = (mainWindow, cstrName) => void 0 !== mainWindow[cstrName] ? new mainWindow[cstrName](noop) : 0; + const addGlobalConstructorUsingPrototype = ($interfaces$, mainWindow, cstrName) => { + void 0 !== mainWindow[cstrName] && $interfaces$.push([ cstrName, "Object", Object.keys(mainWindow[cstrName].prototype).map((propName => [ propName, 6 ])), 12 ]); + }; + let worker; + (receiveMessage => { + const swContainer = window.navigator.serviceWorker; + return swContainer.getRegistration().then((swRegistration => { + swContainer.addEventListener("message", (ev => receiveMessage(ev.data, (accessRsp => swRegistration.active && swRegistration.active.postMessage(accessRsp))))); + return (worker, msg) => { + 0 === msg[0] ? worker.postMessage([ 1, readMainPlatform() ]) : 2 === msg[0] ? worker.postMessage([ 3, readMainInterfaces() ]) : onMessageFromWebWorker(worker, msg); + }; + })); + })(((accessReq, responseCallback) => mainAccessHandler(worker, accessReq).then(responseCallback))).then((onMessageHandler => { + if (onMessageHandler) { + worker = new Worker(libPath + "partytown-ww-sw.js?v=0.5.4", { + name: "Partytown 🎉" + }); + worker.onmessage = ev => { + const msg = ev.data; + 12 === msg[0] ? mainAccessHandler(worker, msg[1]) : onMessageHandler(worker, msg); + }; + logMain("Created Partytown web worker (0.5.4)"); + worker.onerror = ev => console.error("Web Worker Error", ev); + mainWindow.addEventListener("pt1", (ev => registerWindow(worker, getAndSetInstanceId(ev.detail.frameElement), ev.detail))); + } + })); +})(window); diff --git a/website/public/~partytown/debug/partytown-sw.js b/website/public/~partytown/debug/partytown-sw.js new file mode 100644 index 0000000000..14b09e1195 --- /dev/null +++ b/website/public/~partytown/debug/partytown-sw.js @@ -0,0 +1,59 @@ +/* Partytown 0.5.4 - MIT builder.io */ +const resolves = new Map; + +const swMessageError = (accessReq, $error$) => ({ + $msgId$: accessReq.$msgId$, + $error$: $error$ +}); + +const httpRequestFromWebWorker = req => new Promise((async resolve => { + const accessReq = await req.clone().json(); + const responseData = await (accessReq => new Promise((async resolve => { + const clients = await self.clients.matchAll(); + const client = [ ...clients ].sort(((a, b) => a.url > b.url ? -1 : a.url < b.url ? 1 : 0))[0]; + if (client) { + const timeout = 12e4; + const msgResolve = [ resolve, setTimeout((() => { + resolves.delete(accessReq.$msgId$); + resolve(swMessageError(accessReq, "Timeout")); + }), timeout) ]; + resolves.set(accessReq.$msgId$, msgResolve); + client.postMessage(accessReq); + } else { + resolve(swMessageError(accessReq, "NoParty")); + } + })))(accessReq); + resolve(response(JSON.stringify(responseData), "application/json")); +})); + +const response = (body, contentType) => new Response(body, { + headers: { + "content-type": contentType || "text/html", + "Cache-Control": "no-store" + } +}); + +self.oninstall = () => self.skipWaiting(); + +self.onactivate = () => self.clients.claim(); + +self.onmessage = ev => { + const accessRsp = ev.data; + const r = resolves.get(accessRsp.$msgId$); + if (r) { + resolves.delete(accessRsp.$msgId$); + clearTimeout(r[1]); + r[0](accessRsp); + } +}; + +self.onfetch = ev => { + const req = ev.request; + const url = new URL(req.url); + const pathname = url.pathname; + if (pathname.endsWith("sw.html")) { + ev.respondWith(response(' + + + + + + + +
+

+ + +

+ +
+
+
    + +
  • + +
    发布
    +
    &#xe65a;
    +
  • + +
  • + +
    审核
    +
    &#xe697;
    +
  • + +
  • + +
    复制
    +
    &#xe62b;
    +
  • + +
  • + +
    重置密码
    +
    &#xe61a;
    +
  • + +
  • + +
    save
    +
    &#xe6ed;
    +
  • + +
  • + +
    打印
    +
    &#xe620;
    +
  • + +
  • + +
    箭头
    +
    &#xe619;
    +
  • + +
  • + +
    短信
    +
    &#xe680;
    +
  • + +
  • + +
    管理
    +
    &#xe60a;
    +
  • + +
  • + +
    银行卡管理
    +
    &#xe638;
    +
  • + +
  • + +
    电子证件
    +
    &#xe6a5;
    +
  • + +
  • + +
    呼叫拨号
    +
    &#xe744;
    +
  • + +
  • + +
    icon-check
    +
    &#xe60d;
    +
  • + +
  • + +
    icon-shenhe
    +
    &#xe60e;
    +
  • + +
  • + +
    icon-manager
    +
    &#xe60f;
    +
  • + +
  • + +
    icon-shenbao
    +
    &#xe617;
    +
  • + +
  • + +
    icon-jianguan
    +
    &#xe618;
    +
  • + +
  • + +
    刷新
    +
    &#xec08;
    +
  • + +
  • + +
    眼睛_隐藏
    +
    &#xeb45;
    +
  • + +
  • + +
    眼睛_显示
    +
    &#xeb46;
    +
  • + +
  • + +
    返回
    +
    &#xe6c3;
    +
  • + +
  • + +
    编辑 (1)
    +
    &#xe609;
    +
  • + +
  • + +
    删除
    +
    &#xe7c8;
    +
  • + +
  • + +
    登录
    +
    &#xe63a;
    +
  • + +
  • + +
    表单
    +
    &#xe656;
    +
  • + +
  • + +
    密码
    +
    &#xe60c;
    +
  • + +
  • + +
    组件
    +
    &#xe652;
    +
  • + +
  • + +
    功能设置
    +
    &#xe625;
    +
  • + +
  • + +
    电脑
    +
    &#xe6bc;
    +
  • + +
  • + +
    gy组织管理
    +
    &#xe606;
    +
  • + +
  • + +
    系统 设置
    +
    &#xe722;
    +
  • + +
  • + +
    数据
    +
    &#xe608;
    +
  • + +
  • + +
    组件
    +
    &#xe7c7;
    +
  • + +
  • + +
    验证码
    +
    &#xe615;
    +
  • + +
  • + +
    文件夹-展开
    +
    &#xe68d;
    +
  • + +
  • + +
    文件夹-收起
    +
    &#xe68e;
    +
  • + +
  • + +
    文件
    +
    &#xe64f;
    +
  • + +
  • + +
    清除
    +
    &#xe62f;
    +
  • + +
  • + +
    审核
    +
    &#xe65b;
    +
  • + +
  • + +
    all_filter
    +
    &#xe604;
    +
  • + +
  • + +
    expand
    +
    &#xe63e;
    +
  • + +
  • + +
    retract
    +
    &#xe643;
    +
  • + +
  • + +
    cc-check-circle
    +
    &#xe621;
    +
  • + +
  • + +
    delete
    +
    &#xe607;
    +
  • + +
  • + +
    delete
    +
    &#xe614;
    +
  • + +
  • + +
    分期-审核中
    +
    &#xe650;
    +
  • + +
  • + +
    hourglass-o
    +
    &#xe73c;
    +
  • + +
  • + +
    审核
    +
    &#xe603;
    +
  • + +
  • + +
    down
    +
    &#xe613;
    +
  • + +
  • + +
    fold
    +
    &#xe61d;
    +
  • + +
  • + +
    information
    +
    &#xe627;
    +
  • + +
  • + +
    right
    +
    &#xe642;
    +
  • + +
  • + +
    round_check
    +
    &#xe646;
    +
  • + +
  • + +
    +
    &#xe610;
    +
  • + +
  • + +
    方形未选中
    +
    &#xe720;
    +
  • + +
  • + +
    方形选中-fill
    +
    &#xe721;
    +
  • + +
  • + +
    圆形未选中
    +
    &#xe72f;
    +
  • + +
  • + +
    向上1
    +
    &#xe76d;
    +
  • + +
  • + +
    向下1
    +
    &#xe771;
    +
  • + +
  • + +
    提示
    +
    &#xe605;
    +
  • + +
  • + +
    password
    +
    &#xe6a9;
    +
  • + +
  • + +
    question
    +
    &#xe649;
    +
  • + +
  • + +
    top
    +
    &#xe648;
    +
  • + +
  • + +
    收藏1
    +
    &#xe66b;
    +
  • + +
  • + +
    download
    +
    &#xe602;
    +
  • + +
  • + +
    left
    +
    &#xe612;
    +
  • + +
  • + +
    person
    +
    &#xe611;
    +
  • + +
  • + +
    upload
    +
    &#xe666;
    +
  • + +
  • + +
    日期
    +
    &#xe65e;
    +
  • + +
  • + +
    clock
    +
    &#xe624;
    +
  • + +
  • + +
    提示
    +
    &#xe661;
    +
  • + +
  • + +
    提示-fill
    +
    &#xe662;
    +
  • + +
  • + +
    关闭2-fill
    +
    &#xe664;
    +
  • + +
  • + +
    圆形选中-fill
    +
    &#xe668;
    +
  • + +
  • + +
    关闭1
    +
    &#xe66c;
    +
  • + +
  • + +
    右箭头
    +
    &#xe600;
    +
  • + +
  • + +
    左箭头
    +
    &#xe601;
    +
  • + +
  • + +
    收藏
    +
    &#xe670;
    +
  • + +
  • + +
    新增
    +
    &#xe616;
    +
  • + +
  • + +
    查询
    +
    &#xe60b;
    +
  • + +
  • + +
    删除-方框-填充
    +
    &#xe791;
    +
  • + +
  • + +
    unfold
    +
    &#xe61f;
    +
  • + +
+
+

Unicode 引用

+
+ +

Unicode 是字体在网页端最原始的应用方式,特点是:

+
    +
  • 支持按字体的方式去动态调整图标大小,颜色等等。
  • +
  • 默认情况下不支持多色,直接添加多色图标会自动去色。
  • +
+
+

注意:新版 iconfont 支持两种方式引用多色图标:SVG symbol 引用方式和彩色字体图标模式。(使用彩色字体图标需要在「编辑项目」中开启「彩色」选项后并重新生成。)

+
+

Unicode 使用步骤如下:

+

第一步:拷贝项目下面生成的 @font-face

+
@font-face {
+  font-family: 'iconfont';
+  src: url('iconfont.woff2?t=1654879012633') format('woff2'),
+       url('iconfont.woff?t=1654879012633') format('woff'),
+       url('iconfont.ttf?t=1654879012633') format('truetype');
+}
+
+

第二步:定义使用 iconfont 的样式

+
.iconfont {
+  font-family: "iconfont" !important;
+  font-size: 16px;
+  font-style: normal;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+}
+
+

第三步:挑选相应图标并获取字体编码,应用于页面

+
+<span class="iconfont">&#x33;</span>
+
+
+

"iconfont" 是你项目下的 font-family。可以通过编辑项目查看,默认是 "iconfont"。

+
+
+
+
+
    + +
  • + +
    + 发布 +
    +
    .icon-fabu +
    +
  • + +
  • + +
    + 审核 +
    +
    .icon-shenhe2 +
    +
  • + +
  • + +
    + 复制 +
    +
    .icon-fuzhi +
    +
  • + +
  • + +
    + 重置密码 +
    +
    .icon-zhongzhimima +
    +
  • + +
  • + +
    + save +
    +
    .icon-save +
    +
  • + +
  • + +
    + 打印 +
    +
    .icon-dayin +
    +
  • + +
  • + +
    + 箭头 +
    +
    .icon-icon-test +
    +
  • + +
  • + +
    + 短信 +
    +
    .icon-email +
    +
  • + +
  • + +
    + 管理 +
    +
    .icon-coreManage +
    +
  • + +
  • + +
    + 银行卡管理 +
    +
    .icon-cardManage +
    +
  • + +
  • + +
    + 电子证件 +
    +
    .icon-shebaoka +
    +
  • + +
  • + +
    + 呼叫拨号 +
    +
    .icon-call +
    +
  • + +
  • + +
    + icon-check +
    +
    .icon-check +
    +
  • + +
  • + +
    + icon-shenhe +
    +
    .icon-shenhe1 +
    +
  • + +
  • + +
    + icon-manager +
    +
    .icon-manager +
    +
  • + +
  • + +
    + icon-shenbao +
    +
    .icon-shenbao +
    +
  • + +
  • + +
    + icon-jianguan +
    +
    .icon-jianguan +
    +
  • + +
  • + +
    + 刷新 +
    +
    .icon-refresh +
    +
  • + +
  • + +
    + 眼睛_隐藏 +
    +
    .icon-closeEye +
    +
  • + +
  • + +
    + 眼睛_显示 +
    +
    .icon-openEye +
    +
  • + +
  • + +
    + 返回 +
    +
    .icon-return +
    +
  • + +
  • + +
    + 编辑 (1) +
    +
    .icon-edit +
    +
  • + +
  • + +
    + 删除 +
    +
    .icon-delete +
    +
  • + +
  • + +
    + 登录 +
    +
    .icon-loginFun +
    +
  • + +
  • + +
    + 表单 +
    +
    .icon-formFun +
    +
  • + +
  • + +
    + 密码 +
    +
    .icon-pxwFun +
    +
  • + +
  • + +
    + 组件 +
    +
    .icon-zujian +
    +
  • + +
  • + +
    + 功能设置 +
    +
    .icon-funIcon +
    +
  • + +
  • + +
    + 电脑 +
    +
    .icon-systemFun +
    +
  • + +
  • + +
    + gy组织管理 +
    +
    .icon-treeFun +
    +
  • + +
  • + +
    + 系统 设置 +
    +
    .icon-systemSettingFun +
    +
  • + +
  • + +
    + 数据 +
    +
    .icon-tableFun +
    +
  • + +
  • + +
    + 组件 +
    +
    .icon-zujian1 +
    +
  • + +
  • + +
    + 验证码 +
    +
    .icon-identifyCode +
    +
  • + +
  • + +
    + 文件夹-展开 +
    +
    .icon-file-expand +
    +
  • + +
  • + +
    + 文件夹-收起 +
    +
    .icon-file +
    +
  • + +
  • + +
    + 文件 +
    +
    .icon-doc +
    +
  • + +
  • + +
    + 清除 +
    +
    .icon-clean +
    +
  • + +
  • + +
    + 审核 +
    +
    .icon-shenhe +
    +
  • + +
  • + +
    + all_filter +
    +
    .icon-filter +
    +
  • + +
  • + +
    + expand +
    +
    .icon-expand +
    +
  • + +
  • + +
    + retract +
    +
    .icon-retract +
    +
  • + +
  • + +
    + cc-check-circle +
    +
    .icon-cc-check-circle +
    +
  • + +
  • + +
    + delete +
    +
    .icon-error-line +
    +
  • + +
  • + +
    + delete +
    +
    .icon-stop +
    +
  • + +
  • + +
    + 分期-审核中 +
    +
    .icon-more +
    +
  • + +
  • + +
    + hourglass-o +
    +
    .icon-hourglass +
    +
  • + +
  • + +
    + 审核 +
    +
    .icon-verify +
    +
  • + +
  • + +
    + down +
    +
    .icon-download +
    +
  • + +
  • + +
    + fold +
    +
    .icon-up-line +
    +
  • + +
  • + +
    + information +
    +
    .icon-info-round +
    +
  • + +
  • + +
    + right +
    +
    .icon-right-line +
    +
  • + +
  • + +
    + round_check +
    +
    .icon-correct-round +
    +
  • + +
  • + +
    + 勾 +
    +
    .icon-correct +
    +
  • + +
  • + +
    + 方形未选中 +
    +
    .icon-checkbox-uncheck +
    +
  • + +
  • + +
    + 方形选中-fill +
    +
    .icon-checkbox-checked +
    +
  • + +
  • + +
    + 圆形未选中 +
    +
    .icon-radio-unchoose +
    +
  • + +
  • + +
    + 向上1 +
    +
    .icon-up-fill +
    +
  • + +
  • + +
    + 向下1 +
    +
    .icon-down-fill +
    +
  • + +
  • + +
    + 提示 +
    +
    .icon-warn-fill +
    +
  • + +
  • + +
    + password +
    +
    .icon-password +
    +
  • + +
  • + +
    + question +
    +
    .icon-question +
    +
  • + +
  • + +
    + top +
    +
    .icon-top +
    +
  • + +
  • + +
    + 收藏1 +
    +
    .icon-star-line +
    +
  • + +
  • + +
    + download +
    +
    .icon-download-cloud +
    +
  • + +
  • + +
    + left +
    +
    .icon-left-line +
    +
  • + +
  • + +
    + person +
    +
    .icon-account +
    +
  • + +
  • + +
    + upload +
    +
    .icon-upload-cloud +
    +
  • + +
  • + +
    + 日期 +
    +
    .icon-calendar +
    +
  • + +
  • + +
    + clock +
    +
    .icon-clock +
    +
  • + +
  • + +
    + 提示 +
    +
    .icon-info-line +
    +
  • + +
  • + +
    + 提示-fill +
    +
    .icon-info-fill +
    +
  • + +
  • + +
    + 关闭2-fill +
    +
    .icon-error +
    +
  • + +
  • + +
    + 圆形选中-fill +
    +
    .icon-radio-choosed +
    +
  • + +
  • + +
    + 关闭1 +
    +
    .icon-close +
    +
  • + +
  • + +
    + 右箭头 +
    +
    .icon-right-fill +
    +
  • + +
  • + +
    + 左箭头 +
    +
    .icon-left-fill +
    +
  • + +
  • + +
    + 收藏 +
    +
    .icon-start-fill +
    +
  • + +
  • + +
    + 新增 +
    +
    .icon-add +
    +
  • + +
  • + +
    + 查询 +
    +
    .icon-search +
    +
  • + +
  • + +
    + 删除-方框-填充 +
    +
    .icon-checkbox-part +
    +
  • + +
  • + +
    + unfold +
    +
    .icon-down-line +
    +
  • + +
+
+

font-class 引用

+
+ +

font-class 是 Unicode 使用方式的一种变种,主要是解决 Unicode 书写不直观,语意不明确的问题。

+

与 Unicode 使用方式相比,具有如下特点:

+
    +
  • 相比于 Unicode 语意明确,书写更直观。可以很容易分辨这个 icon 是什么。
  • +
  • 因为使用 class 来定义图标,所以当要替换图标时,只需要修改 class 里面的 Unicode 引用。
  • +
+

使用步骤如下:

+

第一步:引入项目下面生成的 fontclass 代码:

+
<link rel="stylesheet" href="./iconfont.css">
+
+

第二步:挑选相应图标并获取类名,应用于页面:

+
<span class="iconfont icon-xxx"></span>
+
+
+

" + iconfont" 是你项目下的 font-family。可以通过编辑项目查看,默认是 "iconfont"。

+
+
+
+
+
    + +
  • + +
    发布
    +
    #icon-fabu
    +
  • + +
  • + +
    审核
    +
    #icon-shenhe2
    +
  • + +
  • + +
    复制
    +
    #icon-fuzhi
    +
  • + +
  • + +
    重置密码
    +
    #icon-zhongzhimima
    +
  • + +
  • + +
    save
    +
    #icon-save
    +
  • + +
  • + +
    打印
    +
    #icon-dayin
    +
  • + +
  • + +
    箭头
    +
    #icon-icon-test
    +
  • + +
  • + +
    短信
    +
    #icon-email
    +
  • + +
  • + +
    管理
    +
    #icon-coreManage
    +
  • + +
  • + +
    银行卡管理
    +
    #icon-cardManage
    +
  • + +
  • + +
    电子证件
    +
    #icon-shebaoka
    +
  • + +
  • + +
    呼叫拨号
    +
    #icon-call
    +
  • + +
  • + +
    icon-check
    +
    #icon-check
    +
  • + +
  • + +
    icon-shenhe
    +
    #icon-shenhe1
    +
  • + +
  • + +
    icon-manager
    +
    #icon-manager
    +
  • + +
  • + +
    icon-shenbao
    +
    #icon-shenbao
    +
  • + +
  • + +
    icon-jianguan
    +
    #icon-jianguan
    +
  • + +
  • + +
    刷新
    +
    #icon-refresh
    +
  • + +
  • + +
    眼睛_隐藏
    +
    #icon-closeEye
    +
  • + +
  • + +
    眼睛_显示
    +
    #icon-openEye
    +
  • + +
  • + +
    返回
    +
    #icon-return
    +
  • + +
  • + +
    编辑 (1)
    +
    #icon-edit
    +
  • + +
  • + +
    删除
    +
    #icon-delete
    +
  • + +
  • + +
    登录
    +
    #icon-loginFun
    +
  • + +
  • + +
    表单
    +
    #icon-formFun
    +
  • + +
  • + +
    密码
    +
    #icon-pxwFun
    +
  • + +
  • + +
    组件
    +
    #icon-zujian
    +
  • + +
  • + +
    功能设置
    +
    #icon-funIcon
    +
  • + +
  • + +
    电脑
    +
    #icon-systemFun
    +
  • + +
  • + +
    gy组织管理
    +
    #icon-treeFun
    +
  • + +
  • + +
    系统 设置
    +
    #icon-systemSettingFun
    +
  • + +
  • + +
    数据
    +
    #icon-tableFun
    +
  • + +
  • + +
    组件
    +
    #icon-zujian1
    +
  • + +
  • + +
    验证码
    +
    #icon-identifyCode
    +
  • + +
  • + +
    文件夹-展开
    +
    #icon-file-expand
    +
  • + +
  • + +
    文件夹-收起
    +
    #icon-file
    +
  • + +
  • + +
    文件
    +
    #icon-doc
    +
  • + +
  • + +
    清除
    +
    #icon-clean
    +
  • + +
  • + +
    审核
    +
    #icon-shenhe
    +
  • + +
  • + +
    all_filter
    +
    #icon-filter
    +
  • + +
  • + +
    expand
    +
    #icon-expand
    +
  • + +
  • + +
    retract
    +
    #icon-retract
    +
  • + +
  • + +
    cc-check-circle
    +
    #icon-cc-check-circle
    +
  • + +
  • + +
    delete
    +
    #icon-error-line
    +
  • + +
  • + +
    delete
    +
    #icon-stop
    +
  • + +
  • + +
    分期-审核中
    +
    #icon-more
    +
  • + +
  • + +
    hourglass-o
    +
    #icon-hourglass
    +
  • + +
  • + +
    审核
    +
    #icon-verify
    +
  • + +
  • + +
    down
    +
    #icon-download
    +
  • + +
  • + +
    fold
    +
    #icon-up-line
    +
  • + +
  • + +
    information
    +
    #icon-info-round
    +
  • + +
  • + +
    right
    +
    #icon-right-line
    +
  • + +
  • + +
    round_check
    +
    #icon-correct-round
    +
  • + +
  • + +
    +
    #icon-correct
    +
  • + +
  • + +
    方形未选中
    +
    #icon-checkbox-uncheck
    +
  • + +
  • + +
    方形选中-fill
    +
    #icon-checkbox-checked
    +
  • + +
  • + +
    圆形未选中
    +
    #icon-radio-unchoose
    +
  • + +
  • + +
    向上1
    +
    #icon-up-fill
    +
  • + +
  • + +
    向下1
    +
    #icon-down-fill
    +
  • + +
  • + +
    提示
    +
    #icon-warn-fill
    +
  • + +
  • + +
    password
    +
    #icon-password
    +
  • + +
  • + +
    question
    +
    #icon-question
    +
  • + +
  • + +
    top
    +
    #icon-top
    +
  • + +
  • + +
    收藏1
    +
    #icon-star-line
    +
  • + +
  • + +
    download
    +
    #icon-download-cloud
    +
  • + +
  • + +
    left
    +
    #icon-left-line
    +
  • + +
  • + +
    person
    +
    #icon-account
    +
  • + +
  • + +
    upload
    +
    #icon-upload-cloud
    +
  • + +
  • + +
    日期
    +
    #icon-calendar
    +
  • + +
  • + +
    clock
    +
    #icon-clock
    +
  • + +
  • + +
    提示
    +
    #icon-info-line
    +
  • + +
  • + +
    提示-fill
    +
    #icon-info-fill
    +
  • + +
  • + +
    关闭2-fill
    +
    #icon-error
    +
  • + +
  • + +
    圆形选中-fill
    +
    #icon-radio-choosed
    +
  • + +
  • + +
    关闭1
    +
    #icon-close
    +
  • + +
  • + +
    右箭头
    +
    #icon-right-fill
    +
  • + +
  • + +
    左箭头
    +
    #icon-left-fill
    +
  • + +
  • + +
    收藏
    +
    #icon-start-fill
    +
  • + +
  • + +
    新增
    +
    #icon-add
    +
  • + +
  • + +
    查询
    +
    #icon-search
    +
  • + +
  • + +
    删除-方框-填充
    +
    #icon-checkbox-part
    +
  • + +
  • + +
    unfold
    +
    #icon-down-line
    +
  • + +
+
+

Symbol 引用

+
+ +

这是一种全新的使用方式,应该说这才是未来的主流,也是平台目前推荐的用法。相关介绍可以参考这篇文章 + 这种用法其实是做了一个 SVG 的集合,与另外两种相比具有如下特点:

+
    +
  • 支持多色图标了,不再受单色限制。
  • +
  • 通过一些技巧,支持像字体那样,通过 font-size, color 来调整样式。
  • +
  • 兼容性较差,支持 IE9+,及现代浏览器。
  • +
  • 浏览器渲染 SVG 的性能一般,还不如 png。
  • +
+

使用步骤如下:

+

第一步:引入项目下面生成的 symbol 代码:

+
<script src="./iconfont.js"></script>
+
+

第二步:加入通用 CSS 代码(引入一次就行):

+
<style>
+.icon {
+  width: 1em;
+  height: 1em;
+  vertical-align: -0.15em;
+  fill: currentColor;
+  overflow: hidden;
+}
+</style>
+
+

第三步:挑选相应图标并获取类名,应用于页面:

+
<svg class="icon" aria-hidden="true">
+  <use xlink:href="#icon-xxx"></use>
+</svg>
+
+
+
+ +
+
+ + + diff --git a/website/src/font/iconfont.css b/website/src/font/iconfont.css new file mode 100644 index 0000000000..70b87be225 --- /dev/null +++ b/website/src/font/iconfont.css @@ -0,0 +1,343 @@ +@font-face { + font-family: "iconfont"; /* Project id 836018 */ + src: url('iconfont.woff2?t=1654879012633') format('woff2'), + url('iconfont.woff?t=1654879012633') format('woff'), + url('iconfont.ttf?t=1654879012633') format('truetype'); +} + +.iconfont { + font-family: "iconfont" !important; + font-size: 16px; + font-style: normal; + -webkit-font-smoothing: antialiased; + -moz-osx-font-smoothing: grayscale; +} + +.icon-fabu:before { + content: "\e65a"; +} + +.icon-shenhe2:before { + content: "\e697"; +} + +.icon-fuzhi:before { + content: "\e62b"; +} + +.icon-zhongzhimima:before { + content: "\e61a"; +} + +.icon-save:before { + content: "\e6ed"; +} + +.icon-dayin:before { + content: "\e620"; +} + +.icon-icon-test:before { + content: "\e619"; +} + +.icon-email:before { + content: "\e680"; +} + +.icon-coreManage:before { + content: "\e60a"; +} + +.icon-cardManage:before { + content: "\e638"; +} + +.icon-shebaoka:before { + content: "\e6a5"; +} + +.icon-call:before { + content: "\e744"; +} + +.icon-check:before { + content: "\e60d"; +} + +.icon-shenhe1:before { + content: "\e60e"; +} + +.icon-manager:before { + content: "\e60f"; +} + +.icon-shenbao:before { + content: "\e617"; +} + +.icon-jianguan:before { + content: "\e618"; +} + +.icon-refresh:before { + content: "\ec08"; +} + +.icon-closeEye:before { + content: "\eb45"; +} + +.icon-openEye:before { + content: "\eb46"; +} + +.icon-return:before { + content: "\e6c3"; +} + +.icon-edit:before { + content: "\e609"; +} + +.icon-delete:before { + content: "\e7c8"; +} + +.icon-loginFun:before { + content: "\e63a"; +} + +.icon-formFun:before { + content: "\e656"; +} + +.icon-pxwFun:before { + content: "\e60c"; +} + +.icon-zujian:before { + content: "\e652"; +} + +.icon-funIcon:before { + content: "\e625"; +} + +.icon-systemFun:before { + content: "\e6bc"; +} + +.icon-treeFun:before { + content: "\e606"; +} + +.icon-systemSettingFun:before { + content: "\e722"; +} + +.icon-tableFun:before { + content: "\e608"; +} + +.icon-zujian1:before { + content: "\e7c7"; +} + +.icon-identifyCode:before { + content: "\e615"; +} + +.icon-file-expand:before { + content: "\e68d"; +} + +.icon-file:before { + content: "\e68e"; +} + +.icon-doc:before { + content: "\e64f"; +} + +.icon-clean:before { + content: "\e62f"; +} + +.icon-shenhe:before { + content: "\e65b"; +} + +.icon-filter:before { + content: "\e604"; +} + +.icon-expand:before { + content: "\e63e"; +} + +.icon-retract:before { + content: "\e643"; +} + +.icon-cc-check-circle:before { + content: "\e621"; +} + +.icon-error-line:before { + content: "\e607"; +} + +.icon-stop:before { + content: "\e614"; +} + +.icon-more:before { + content: "\e650"; +} + +.icon-hourglass:before { + content: "\e73c"; +} + +.icon-verify:before { + content: "\e603"; +} + +.icon-download:before { + content: "\e613"; +} + +.icon-up-line:before { + content: "\e61d"; +} + +.icon-info-round:before { + content: "\e627"; +} + +.icon-right-line:before { + content: "\e642"; +} + +.icon-correct-round:before { + content: "\e646"; +} + +.icon-correct:before { + content: "\e610"; +} + +.icon-checkbox-uncheck:before { + content: "\e720"; +} + +.icon-checkbox-checked:before { + content: "\e721"; +} + +.icon-radio-unchoose:before { + content: "\e72f"; +} + +.icon-up-fill:before { + content: "\e76d"; +} + +.icon-down-fill:before { + content: "\e771"; +} + +.icon-warn-fill:before { + content: "\e605"; +} + +.icon-password:before { + content: "\e6a9"; +} + +.icon-question:before { + content: "\e649"; +} + +.icon-top:before { + content: "\e648"; +} + +.icon-star-line:before { + content: "\e66b"; +} + +.icon-download-cloud:before { + content: "\e602"; +} + +.icon-left-line:before { + content: "\e612"; +} + +.icon-account:before { + content: "\e611"; +} + +.icon-upload-cloud:before { + content: "\e666"; +} + +.icon-calendar:before { + content: "\e65e"; +} + +.icon-clock:before { + content: "\e624"; +} + +.icon-info-line:before { + content: "\e661"; +} + +.icon-info-fill:before { + content: "\e662"; +} + +.icon-error:before { + content: "\e664"; +} + +.icon-radio-choosed:before { + content: "\e668"; +} + +.icon-close:before { + content: "\e66c"; +} + +.icon-right-fill:before { + content: "\e600"; +} + +.icon-left-fill:before { + content: "\e601"; +} + +.icon-start-fill:before { + content: "\e670"; +} + +.icon-add:before { + content: "\e616"; +} + +.icon-search:before { + content: "\e60b"; +} + +.icon-checkbox-part:before { + content: "\e791"; +} + +.icon-down-line:before { + content: "\e61f"; +} + diff --git a/website/src/font/iconfont.js b/website/src/font/iconfont.js new file mode 100644 index 0000000000..6b6c9a4e0f --- /dev/null +++ b/website/src/font/iconfont.js @@ -0,0 +1 @@ +!function(c){var l,a,h,t,o,i='',s=(s=document.getElementsByTagName("script"))[s.length-1].getAttribute("data-injectcss"),p=function(c,l){l.parentNode.insertBefore(c,l)};if(s&&!c.__iconfont__svg__cssinject__){c.__iconfont__svg__cssinject__=!0;try{document.write("")}catch(c){console&&console.log(c)}}function d(){o||(o=!0,h())}function e(){try{t.documentElement.doScroll("left")}catch(c){return void setTimeout(e,50)}d()}l=function(){var c,l=document.createElement("div");l.innerHTML=i,i=null,(l=l.getElementsByTagName("svg")[0])&&(l.setAttribute("aria-hidden","true"),l.style.position="absolute",l.style.width=0,l.style.height=0,l.style.overflow="hidden",l=l,(c=document.body).firstChild?p(l,c.firstChild):c.appendChild(l))},document.addEventListener?~["complete","loaded","interactive"].indexOf(document.readyState)?setTimeout(l,0):(a=function(){document.removeEventListener("DOMContentLoaded",a,!1),l()},document.addEventListener("DOMContentLoaded",a,!1)):document.attachEvent&&(h=l,t=c.document,o=!1,e(),t.onreadystatechange=function(){"complete"==t.readyState&&(t.onreadystatechange=null,d())})}(window); \ No newline at end of file diff --git a/website/src/font/iconfont.json b/website/src/font/iconfont.json new file mode 100644 index 0000000000..0a3ad760e4 --- /dev/null +++ b/website/src/font/iconfont.json @@ -0,0 +1,583 @@ +{ + "id": "836018", + "name": "tcomponent", + "font_family": "iconfont", + "css_prefix_text": "icon-", + "description": "", + "glyphs": [ + { + "icon_id": "2800076", + "name": "发布", + "font_class": "fabu", + "unicode": "e65a", + "unicode_decimal": 58970 + }, + { + "icon_id": "17029934", + "name": "审核", + "font_class": "shenhe2", + "unicode": "e697", + "unicode_decimal": 59031 + }, + { + "icon_id": "12550395", + "name": "复制", + "font_class": "fuzhi", + "unicode": "e62b", + "unicode_decimal": 58923 + }, + { + "icon_id": "524416", + "name": "重置密码", + "font_class": "zhongzhimima", + "unicode": "e61a", + "unicode_decimal": 58906 + }, + { + "icon_id": "2285753", + "name": "save", + "font_class": "save", + "unicode": "e6ed", + "unicode_decimal": 59117 + }, + { + "icon_id": "3278356", + "name": "打印", + "font_class": "dayin", + "unicode": "e620", + "unicode_decimal": 58912 + }, + { + "icon_id": "5609243", + "name": "箭头", + "font_class": "icon-test", + "unicode": "e619", + "unicode_decimal": 58905 + }, + { + "icon_id": "4195151", + "name": "短信", + "font_class": "email", + "unicode": "e680", + "unicode_decimal": 59008 + }, + { + "icon_id": "1190", + "name": "管理", + "font_class": "coreManage", + "unicode": "e60a", + "unicode_decimal": 58890 + }, + { + "icon_id": "677607", + "name": "银行卡管理", + "font_class": "cardManage", + "unicode": "e638", + "unicode_decimal": 58936 + }, + { + "icon_id": "11177882", + "name": "电子证件", + "font_class": "shebaoka", + "unicode": "e6a5", + "unicode_decimal": 59045 + }, + { + "icon_id": "8908215", + "name": "呼叫拨号", + "font_class": "call", + "unicode": "e744", + "unicode_decimal": 59204 + }, + { + "icon_id": "13307657", + "name": "icon-check", + "font_class": "check", + "unicode": "e60d", + "unicode_decimal": 58893 + }, + { + "icon_id": "13307658", + "name": "icon-shenhe", + "font_class": "shenhe1", + "unicode": "e60e", + "unicode_decimal": 58894 + }, + { + "icon_id": "13307659", + "name": "icon-manager", + "font_class": "manager", + "unicode": "e60f", + "unicode_decimal": 58895 + }, + { + "icon_id": "13307660", + "name": "icon-shenbao", + "font_class": "shenbao", + "unicode": "e617", + "unicode_decimal": 58903 + }, + { + "icon_id": "13307661", + "name": "icon-jianguan", + "font_class": "jianguan", + "unicode": "e618", + "unicode_decimal": 58904 + }, + { + "icon_id": "4686545", + "name": "刷新", + "font_class": "refresh", + "unicode": "ec08", + "unicode_decimal": 60424 + }, + { + "icon_id": "5387735", + "name": "眼睛_隐藏", + "font_class": "closeEye", + "unicode": "eb45", + "unicode_decimal": 60229 + }, + { + "icon_id": "5387736", + "name": "眼睛_显示", + "font_class": "openEye", + "unicode": "eb46", + "unicode_decimal": 60230 + }, + { + "icon_id": "998817", + "name": "返回", + "font_class": "return", + "unicode": "e6c3", + "unicode_decimal": 59075 + }, + { + "icon_id": "887766", + "name": "编辑 (1)", + "font_class": "edit", + "unicode": "e609", + "unicode_decimal": 58889 + }, + { + "icon_id": "12901232", + "name": "删除", + "font_class": "delete", + "unicode": "e7c8", + "unicode_decimal": 59336 + }, + { + "icon_id": "826948", + "name": "登录", + "font_class": "loginFun", + "unicode": "e63a", + "unicode_decimal": 58938 + }, + { + "icon_id": "1526272", + "name": "表单", + "font_class": "formFun", + "unicode": "e656", + "unicode_decimal": 58966 + }, + { + "icon_id": "5485633", + "name": "密码", + "font_class": "pxwFun", + "unicode": "e60c", + "unicode_decimal": 58892 + }, + { + "icon_id": "6246309", + "name": "组件", + "font_class": "zujian", + "unicode": "e652", + "unicode_decimal": 58962 + }, + { + "icon_id": "6842458", + "name": "功能设置", + "font_class": "funIcon", + "unicode": "e625", + "unicode_decimal": 58917 + }, + { + "icon_id": "8361759", + "name": "电脑", + "font_class": "systemFun", + "unicode": "e6bc", + "unicode_decimal": 59068 + }, + { + "icon_id": "9113626", + "name": "gy组织管理", + "font_class": "treeFun", + "unicode": "e606", + "unicode_decimal": 58886 + }, + { + "icon_id": "10809387", + "name": "系统 设置", + "font_class": "systemSettingFun", + "unicode": "e722", + "unicode_decimal": 59170 + }, + { + "icon_id": "11459730", + "name": "数据", + "font_class": "tableFun", + "unicode": "e608", + "unicode_decimal": 58888 + }, + { + "icon_id": "11759988", + "name": "组件", + "font_class": "zujian1", + "unicode": "e7c7", + "unicode_decimal": 59335 + }, + { + "icon_id": "1046777", + "name": "验证码", + "font_class": "identifyCode", + "unicode": "e615", + "unicode_decimal": 58901 + }, + { + "icon_id": "1788486", + "name": "文件夹-展开", + "font_class": "file-expand", + "unicode": "e68d", + "unicode_decimal": 59021 + }, + { + "icon_id": "1788487", + "name": "文件夹-收起", + "font_class": "file", + "unicode": "e68e", + "unicode_decimal": 59022 + }, + { + "icon_id": "6265221", + "name": "文件", + "font_class": "doc", + "unicode": "e64f", + "unicode_decimal": 58959 + }, + { + "icon_id": "7029932", + "name": "清除", + "font_class": "clean", + "unicode": "e62f", + "unicode_decimal": 58927 + }, + { + "icon_id": "623940", + "name": "审核", + "font_class": "shenhe", + "unicode": "e65b", + "unicode_decimal": 58971 + }, + { + "icon_id": "6298812", + "name": "all_filter", + "font_class": "filter", + "unicode": "e604", + "unicode_decimal": 58884 + }, + { + "icon_id": "1074551", + "name": "expand", + "font_class": "expand", + "unicode": "e63e", + "unicode_decimal": 58942 + }, + { + "icon_id": "1074556", + "name": "retract", + "font_class": "retract", + "unicode": "e643", + "unicode_decimal": 58947 + }, + { + "icon_id": "372127", + "name": "cc-check-circle", + "font_class": "cc-check-circle", + "unicode": "e621", + "unicode_decimal": 58913 + }, + { + "icon_id": "438021", + "name": "delete", + "font_class": "error-line", + "unicode": "e607", + "unicode_decimal": 58887 + }, + { + "icon_id": "548073", + "name": "delete", + "font_class": "stop", + "unicode": "e614", + "unicode_decimal": 58900 + }, + { + "icon_id": "657338", + "name": "分期-审核中", + "font_class": "more", + "unicode": "e650", + "unicode_decimal": 58960 + }, + { + "icon_id": "929466", + "name": "hourglass-o", + "font_class": "hourglass", + "unicode": "e73c", + "unicode_decimal": 59196 + }, + { + "icon_id": "3865996", + "name": "审核", + "font_class": "verify", + "unicode": "e603", + "unicode_decimal": 58883 + }, + { + "icon_id": "109732", + "name": "down", + "font_class": "download", + "unicode": "e613", + "unicode_decimal": 58899 + }, + { + "icon_id": "109742", + "name": "fold", + "font_class": "up-line", + "unicode": "e61d", + "unicode_decimal": 58909 + }, + { + "icon_id": "109752", + "name": "information", + "font_class": "info-round", + "unicode": "e627", + "unicode_decimal": 58919 + }, + { + "icon_id": "109779", + "name": "right", + "font_class": "right-line", + "unicode": "e642", + "unicode_decimal": 58946 + }, + { + "icon_id": "109783", + "name": "round_check", + "font_class": "correct-round", + "unicode": "e646", + "unicode_decimal": 58950 + }, + { + "icon_id": "356441", + "name": "勾", + "font_class": "correct", + "unicode": "e610", + "unicode_decimal": 58896 + }, + { + "icon_id": "577306", + "name": "方形未选中", + "font_class": "checkbox-uncheck", + "unicode": "e720", + "unicode_decimal": 59168 + }, + { + "icon_id": "577307", + "name": "方形选中-fill", + "font_class": "checkbox-checked", + "unicode": "e721", + "unicode_decimal": 59169 + }, + { + "icon_id": "577322", + "name": "圆形未选中", + "font_class": "radio-unchoose", + "unicode": "e72f", + "unicode_decimal": 59183 + }, + { + "icon_id": "577394", + "name": "向上1", + "font_class": "up-fill", + "unicode": "e76d", + "unicode_decimal": 59245 + }, + { + "icon_id": "577398", + "name": "向下1", + "font_class": "down-fill", + "unicode": "e771", + "unicode_decimal": 59249 + }, + { + "icon_id": "735255", + "name": "提示", + "font_class": "warn-fill", + "unicode": "e605", + "unicode_decimal": 58885 + }, + { + "icon_id": "851473", + "name": "password", + "font_class": "password", + "unicode": "e6a9", + "unicode_decimal": 59049 + }, + { + "icon_id": "963368", + "name": "question", + "font_class": "question", + "unicode": "e649", + "unicode_decimal": 58953 + }, + { + "icon_id": "1018303", + "name": "top", + "font_class": "top", + "unicode": "e648", + "unicode_decimal": 58952 + }, + { + "icon_id": "1071049", + "name": "收藏1", + "font_class": "star-line", + "unicode": "e66b", + "unicode_decimal": 58987 + }, + { + "icon_id": "1129504", + "name": "download", + "font_class": "download-cloud", + "unicode": "e602", + "unicode_decimal": 58882 + }, + { + "icon_id": "1133224", + "name": "left", + "font_class": "left-line", + "unicode": "e612", + "unicode_decimal": 58898 + }, + { + "icon_id": "1199332", + "name": "person", + "font_class": "account", + "unicode": "e611", + "unicode_decimal": 58897 + }, + { + "icon_id": "1277587", + "name": "upload", + "font_class": "upload-cloud", + "unicode": "e666", + "unicode_decimal": 58982 + }, + { + "icon_id": "1301369", + "name": "日期", + "font_class": "calendar", + "unicode": "e65e", + "unicode_decimal": 58974 + }, + { + "icon_id": "1433682", + "name": "clock", + "font_class": "clock", + "unicode": "e624", + "unicode_decimal": 58916 + }, + { + "icon_id": "1526315", + "name": "提示", + "font_class": "info-line", + "unicode": "e661", + "unicode_decimal": 58977 + }, + { + "icon_id": "1526316", + "name": "提示-fill", + "font_class": "info-fill", + "unicode": "e662", + "unicode_decimal": 58978 + }, + { + "icon_id": "1526319", + "name": "关闭2-fill", + "font_class": "error", + "unicode": "e664", + "unicode_decimal": 58980 + }, + { + "icon_id": "1526323", + "name": "圆形选中-fill", + "font_class": "radio-choosed", + "unicode": "e668", + "unicode_decimal": 58984 + }, + { + "icon_id": "1526329", + "name": "关闭1", + "font_class": "close", + "unicode": "e66c", + "unicode_decimal": 58988 + }, + { + "icon_id": "1718355", + "name": "右箭头", + "font_class": "right-fill", + "unicode": "e600", + "unicode_decimal": 58880 + }, + { + "icon_id": "1718358", + "name": "左箭头", + "font_class": "left-fill", + "unicode": "e601", + "unicode_decimal": 58881 + }, + { + "icon_id": "2418255", + "name": "收藏", + "font_class": "start-fill", + "unicode": "e670", + "unicode_decimal": 58992 + }, + { + "icon_id": "3248452", + "name": "新增", + "font_class": "add", + "unicode": "e616", + "unicode_decimal": 58902 + }, + { + "icon_id": "3977817", + "name": "查询", + "font_class": "search", + "unicode": "e60b", + "unicode_decimal": 58891 + }, + { + "icon_id": "4425815", + "name": "删除-方框-填充", + "font_class": "checkbox-part", + "unicode": "e791", + "unicode_decimal": 59281 + }, + { + "icon_id": "5755908", + "name": "unfold", + "font_class": "down-line", + "unicode": "e61f", + "unicode_decimal": 58911 + } + ] +} diff --git a/website/src/font/iconfont.ttf b/website/src/font/iconfont.ttf new file mode 100644 index 0000000000..c33b6463f2 Binary files /dev/null and b/website/src/font/iconfont.ttf differ diff --git a/website/src/font/iconfont.woff b/website/src/font/iconfont.woff new file mode 100644 index 0000000000..440fffd2b5 Binary files /dev/null and b/website/src/font/iconfont.woff differ diff --git a/website/src/font/iconfont.woff2 b/website/src/font/iconfont.woff2 new file mode 100644 index 0000000000..c3afd96fca Binary files /dev/null and b/website/src/font/iconfont.woff2 differ diff --git a/docs/images/LogMiner/LogMiner1.png b/website/src/images/doc/LogMiner/LogMiner1.png similarity index 100% rename from docs/images/LogMiner/LogMiner1.png rename to website/src/images/doc/LogMiner/LogMiner1.png diff --git a/docs/images/LogMiner/LogMiner10.png b/website/src/images/doc/LogMiner/LogMiner10.png similarity index 100% rename from docs/images/LogMiner/LogMiner10.png rename to website/src/images/doc/LogMiner/LogMiner10.png diff --git a/docs/images/LogMiner/LogMiner11.png b/website/src/images/doc/LogMiner/LogMiner11.png similarity index 100% rename from docs/images/LogMiner/LogMiner11.png rename to website/src/images/doc/LogMiner/LogMiner11.png diff --git a/docs/images/LogMiner/LogMiner12.png b/website/src/images/doc/LogMiner/LogMiner12.png similarity index 100% rename from docs/images/LogMiner/LogMiner12.png rename to website/src/images/doc/LogMiner/LogMiner12.png diff --git a/docs/images/LogMiner/LogMiner13.png b/website/src/images/doc/LogMiner/LogMiner13.png similarity index 100% rename from docs/images/LogMiner/LogMiner13.png rename to website/src/images/doc/LogMiner/LogMiner13.png diff --git a/docs/images/LogMiner/LogMiner14.png b/website/src/images/doc/LogMiner/LogMiner14.png similarity index 100% rename from docs/images/LogMiner/LogMiner14.png rename to website/src/images/doc/LogMiner/LogMiner14.png diff --git a/docs/images/LogMiner/LogMiner15.png b/website/src/images/doc/LogMiner/LogMiner15.png similarity index 100% rename from docs/images/LogMiner/LogMiner15.png rename to website/src/images/doc/LogMiner/LogMiner15.png diff --git a/docs/images/LogMiner/LogMiner16.png b/website/src/images/doc/LogMiner/LogMiner16.png similarity index 100% rename from docs/images/LogMiner/LogMiner16.png rename to website/src/images/doc/LogMiner/LogMiner16.png diff --git a/docs/images/LogMiner/LogMiner17.png b/website/src/images/doc/LogMiner/LogMiner17.png similarity index 100% rename from docs/images/LogMiner/LogMiner17.png rename to website/src/images/doc/LogMiner/LogMiner17.png diff --git a/docs/images/LogMiner/LogMiner18.png b/website/src/images/doc/LogMiner/LogMiner18.png similarity index 100% rename from docs/images/LogMiner/LogMiner18.png rename to website/src/images/doc/LogMiner/LogMiner18.png diff --git a/docs/images/LogMiner/LogMiner19.png b/website/src/images/doc/LogMiner/LogMiner19.png similarity index 100% rename from docs/images/LogMiner/LogMiner19.png rename to website/src/images/doc/LogMiner/LogMiner19.png diff --git a/docs/images/LogMiner/LogMiner2.png b/website/src/images/doc/LogMiner/LogMiner2.png similarity index 100% rename from docs/images/LogMiner/LogMiner2.png rename to website/src/images/doc/LogMiner/LogMiner2.png diff --git a/docs/images/LogMiner/LogMiner20.png b/website/src/images/doc/LogMiner/LogMiner20.png similarity index 100% rename from docs/images/LogMiner/LogMiner20.png rename to website/src/images/doc/LogMiner/LogMiner20.png diff --git a/docs/images/LogMiner/LogMiner21.png b/website/src/images/doc/LogMiner/LogMiner21.png similarity index 100% rename from docs/images/LogMiner/LogMiner21.png rename to website/src/images/doc/LogMiner/LogMiner21.png diff --git a/docs/images/LogMiner/LogMiner22.png b/website/src/images/doc/LogMiner/LogMiner22.png similarity index 100% rename from docs/images/LogMiner/LogMiner22.png rename to website/src/images/doc/LogMiner/LogMiner22.png diff --git a/docs/images/LogMiner/LogMiner23.png b/website/src/images/doc/LogMiner/LogMiner23.png similarity index 100% rename from docs/images/LogMiner/LogMiner23.png rename to website/src/images/doc/LogMiner/LogMiner23.png diff --git a/docs/images/LogMiner/LogMiner3.png b/website/src/images/doc/LogMiner/LogMiner3.png similarity index 100% rename from docs/images/LogMiner/LogMiner3.png rename to website/src/images/doc/LogMiner/LogMiner3.png diff --git a/docs/images/LogMiner/LogMiner4.png b/website/src/images/doc/LogMiner/LogMiner4.png similarity index 100% rename from docs/images/LogMiner/LogMiner4.png rename to website/src/images/doc/LogMiner/LogMiner4.png diff --git a/docs/images/LogMiner/LogMiner5.png b/website/src/images/doc/LogMiner/LogMiner5.png similarity index 100% rename from docs/images/LogMiner/LogMiner5.png rename to website/src/images/doc/LogMiner/LogMiner5.png diff --git a/docs/images/LogMiner/LogMiner6.png b/website/src/images/doc/LogMiner/LogMiner6.png similarity index 100% rename from docs/images/LogMiner/LogMiner6.png rename to website/src/images/doc/LogMiner/LogMiner6.png diff --git a/docs/images/LogMiner/LogMiner7.png b/website/src/images/doc/LogMiner/LogMiner7.png similarity index 100% rename from docs/images/LogMiner/LogMiner7.png rename to website/src/images/doc/LogMiner/LogMiner7.png diff --git a/docs/images/LogMiner/LogMiner8.png b/website/src/images/doc/LogMiner/LogMiner8.png similarity index 100% rename from docs/images/LogMiner/LogMiner8.png rename to website/src/images/doc/LogMiner/LogMiner8.png diff --git a/docs/images/LogMiner/LogMiner9.png b/website/src/images/doc/LogMiner/LogMiner9.png similarity index 100% rename from docs/images/LogMiner/LogMiner9.png rename to website/src/images/doc/LogMiner/LogMiner9.png diff --git a/docs/images/SqlserverCDC/Sqlserver1.png b/website/src/images/doc/SqlserverCDC/Sqlserver1.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver1.png rename to website/src/images/doc/SqlserverCDC/Sqlserver1.png diff --git a/docs/images/SqlserverCDC/Sqlserver10.png b/website/src/images/doc/SqlserverCDC/Sqlserver10.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver10.png rename to website/src/images/doc/SqlserverCDC/Sqlserver10.png diff --git a/docs/images/SqlserverCDC/Sqlserver11.png b/website/src/images/doc/SqlserverCDC/Sqlserver11.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver11.png rename to website/src/images/doc/SqlserverCDC/Sqlserver11.png diff --git a/docs/images/SqlserverCDC/Sqlserver12.png b/website/src/images/doc/SqlserverCDC/Sqlserver12.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver12.png rename to website/src/images/doc/SqlserverCDC/Sqlserver12.png diff --git a/docs/images/SqlserverCDC/Sqlserver13.png b/website/src/images/doc/SqlserverCDC/Sqlserver13.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver13.png rename to website/src/images/doc/SqlserverCDC/Sqlserver13.png diff --git a/docs/images/SqlserverCDC/Sqlserver14.png b/website/src/images/doc/SqlserverCDC/Sqlserver14.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver14.png rename to website/src/images/doc/SqlserverCDC/Sqlserver14.png diff --git a/docs/images/SqlserverCDC/Sqlserver16.png b/website/src/images/doc/SqlserverCDC/Sqlserver16.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver16.png rename to website/src/images/doc/SqlserverCDC/Sqlserver16.png diff --git a/docs/images/SqlserverCDC/Sqlserver17.png b/website/src/images/doc/SqlserverCDC/Sqlserver17.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver17.png rename to website/src/images/doc/SqlserverCDC/Sqlserver17.png diff --git a/docs/images/SqlserverCDC/Sqlserver18.png b/website/src/images/doc/SqlserverCDC/Sqlserver18.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver18.png rename to website/src/images/doc/SqlserverCDC/Sqlserver18.png diff --git a/docs/images/SqlserverCDC/Sqlserver19.png b/website/src/images/doc/SqlserverCDC/Sqlserver19.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver19.png rename to website/src/images/doc/SqlserverCDC/Sqlserver19.png diff --git a/docs/images/SqlserverCDC/Sqlserver2.png b/website/src/images/doc/SqlserverCDC/Sqlserver2.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver2.png rename to website/src/images/doc/SqlserverCDC/Sqlserver2.png diff --git a/docs/images/SqlserverCDC/Sqlserver3.png b/website/src/images/doc/SqlserverCDC/Sqlserver3.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver3.png rename to website/src/images/doc/SqlserverCDC/Sqlserver3.png diff --git a/docs/images/SqlserverCDC/Sqlserver4.png b/website/src/images/doc/SqlserverCDC/Sqlserver4.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver4.png rename to website/src/images/doc/SqlserverCDC/Sqlserver4.png diff --git a/docs/images/SqlserverCDC/Sqlserver5.png b/website/src/images/doc/SqlserverCDC/Sqlserver5.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver5.png rename to website/src/images/doc/SqlserverCDC/Sqlserver5.png diff --git a/docs/images/SqlserverCDC/Sqlserver6.png b/website/src/images/doc/SqlserverCDC/Sqlserver6.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver6.png rename to website/src/images/doc/SqlserverCDC/Sqlserver6.png diff --git a/docs/images/SqlserverCDC/Sqlserver7.png b/website/src/images/doc/SqlserverCDC/Sqlserver7.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver7.png rename to website/src/images/doc/SqlserverCDC/Sqlserver7.png diff --git a/docs/images/SqlserverCDC/Sqlserver8.png b/website/src/images/doc/SqlserverCDC/Sqlserver8.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver8.png rename to website/src/images/doc/SqlserverCDC/Sqlserver8.png diff --git a/docs/images/SqlserverCDC/Sqlserver9.png b/website/src/images/doc/SqlserverCDC/Sqlserver9.png similarity index 100% rename from docs/images/SqlserverCDC/Sqlserver9.png rename to website/src/images/doc/SqlserverCDC/Sqlserver9.png diff --git "a/website/src/images/doc/SqlserverCDC/SqlserverCdc\346\265\201\347\250\213\345\233\276.png" "b/website/src/images/doc/SqlserverCDC/SqlserverCdc\346\265\201\347\250\213\345\233\276.png" new file mode 100644 index 0000000000..2923ee3869 Binary files /dev/null and "b/website/src/images/doc/SqlserverCDC/SqlserverCdc\346\265\201\347\250\213\345\233\276.png" differ diff --git a/website/src/images/doc/contribute/195230-11f4ee6bc7e788c7.webp b/website/src/images/doc/contribute/195230-11f4ee6bc7e788c7.webp new file mode 100644 index 0000000000..28be825950 Binary files /dev/null and b/website/src/images/doc/contribute/195230-11f4ee6bc7e788c7.webp differ diff --git a/website/src/images/doc/contribute/image-20220614143347037.png b/website/src/images/doc/contribute/image-20220614143347037.png new file mode 100644 index 0000000000..d48fc7ae48 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614143347037.png differ diff --git a/website/src/images/doc/contribute/image-20220614144650798.png b/website/src/images/doc/contribute/image-20220614144650798.png new file mode 100644 index 0000000000..9efb233d68 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614144650798.png differ diff --git a/website/src/images/doc/contribute/image-20220614145127874.png b/website/src/images/doc/contribute/image-20220614145127874.png new file mode 100644 index 0000000000..35c51b6972 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614145127874.png differ diff --git a/website/src/images/doc/contribute/image-20220614145839069.png b/website/src/images/doc/contribute/image-20220614145839069.png new file mode 100644 index 0000000000..fd53765145 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614145839069.png differ diff --git a/website/src/images/doc/contribute/image-20220614150742626.png b/website/src/images/doc/contribute/image-20220614150742626.png new file mode 100644 index 0000000000..481a746405 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614150742626.png differ diff --git a/website/src/images/doc/contribute/image-20220614151300210.png b/website/src/images/doc/contribute/image-20220614151300210.png new file mode 100644 index 0000000000..0243f9f6fb Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614151300210.png differ diff --git a/website/src/images/doc/contribute/image-20220614152751238.png b/website/src/images/doc/contribute/image-20220614152751238.png new file mode 100644 index 0000000000..91512881dd Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614152751238.png differ diff --git a/website/src/images/doc/contribute/image-20220614152928836.png b/website/src/images/doc/contribute/image-20220614152928836.png new file mode 100644 index 0000000000..8659ce4b19 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614152928836.png differ diff --git a/website/src/images/doc/contribute/image-20220614153545215.png b/website/src/images/doc/contribute/image-20220614153545215.png new file mode 100644 index 0000000000..e300384439 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614153545215.png differ diff --git a/website/src/images/doc/contribute/image-20220614171917692.png b/website/src/images/doc/contribute/image-20220614171917692.png new file mode 100644 index 0000000000..181e97f0ff Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614171917692.png differ diff --git a/website/src/images/doc/contribute/image-20220614172338108.png b/website/src/images/doc/contribute/image-20220614172338108.png new file mode 100644 index 0000000000..6c71a23fb3 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220614172338108.png differ diff --git a/website/src/images/doc/contribute/image-20220616102414067.png b/website/src/images/doc/contribute/image-20220616102414067.png new file mode 100644 index 0000000000..a4b051f1cf Binary files /dev/null and b/website/src/images/doc/contribute/image-20220616102414067.png differ diff --git a/website/src/images/doc/contribute/image-20220616102856402.png b/website/src/images/doc/contribute/image-20220616102856402.png new file mode 100644 index 0000000000..99760409b3 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220616102856402.png differ diff --git a/website/src/images/doc/contribute/image-20220616103241458.png b/website/src/images/doc/contribute/image-20220616103241458.png new file mode 100644 index 0000000000..486687f962 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220616103241458.png differ diff --git a/website/src/images/doc/contribute/image-20220616112115980.png b/website/src/images/doc/contribute/image-20220616112115980.png new file mode 100644 index 0000000000..53accbc196 Binary files /dev/null and b/website/src/images/doc/contribute/image-20220616112115980.png differ diff --git a/website/src/images/doc/dirty/dirty-1.png b/website/src/images/doc/dirty/dirty-1.png new file mode 100644 index 0000000000..d480dad223 Binary files /dev/null and b/website/src/images/doc/dirty/dirty-1.png differ diff --git a/website/src/images/doc/dirty/dirty-2.png b/website/src/images/doc/dirty/dirty-2.png new file mode 100644 index 0000000000..a041b86c6a Binary files /dev/null and b/website/src/images/doc/dirty/dirty-2.png differ diff --git a/website/src/images/doc/dirty/dirty-conf.png b/website/src/images/doc/dirty/dirty-conf.png new file mode 100644 index 0000000000..8fab211036 Binary files /dev/null and b/website/src/images/doc/dirty/dirty-conf.png differ diff --git a/website/src/images/doc/dirty/dirty-manager.png b/website/src/images/doc/dirty/dirty-manager.png new file mode 100644 index 0000000000..d245ddb7df Binary files /dev/null and b/website/src/images/doc/dirty/dirty-manager.png differ diff --git a/website/src/images/doc/doc.svg b/website/src/images/doc/doc.svg new file mode 100644 index 0000000000..9f3238230f --- /dev/null +++ b/website/src/images/doc/doc.svg @@ -0,0 +1 @@ + diff --git a/website/src/images/doc/docusaurus.png b/website/src/images/doc/docusaurus.png new file mode 100644 index 0000000000..f458149e3c Binary files /dev/null and b/website/src/images/doc/docusaurus.png differ diff --git a/website/src/images/doc/easyuse.svg b/website/src/images/doc/easyuse.svg new file mode 100644 index 0000000000..d849c4bf57 --- /dev/null +++ b/website/src/images/doc/easyuse.svg @@ -0,0 +1 @@ + diff --git a/website/src/images/doc/favicon.ico b/website/src/images/doc/favicon.ico new file mode 100644 index 0000000000..c01d54bcd3 Binary files /dev/null and b/website/src/images/doc/favicon.ico differ diff --git a/website/src/images/doc/flink_header_logo.svg b/website/src/images/doc/flink_header_logo.svg new file mode 100644 index 0000000000..33ba8b7b3d --- /dev/null +++ b/website/src/images/doc/flink_header_logo.svg @@ -0,0 +1,211 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/images/doc/incremental/prometheus-search.png b/website/src/images/doc/incremental/prometheus-search.png new file mode 100644 index 0000000000..34272b44fc Binary files /dev/null and b/website/src/images/doc/incremental/prometheus-search.png differ diff --git a/website/src/images/doc/logo-dark.svg b/website/src/images/doc/logo-dark.svg new file mode 100644 index 0000000000..c0625d0aea --- /dev/null +++ b/website/src/images/doc/logo-dark.svg @@ -0,0 +1,161 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/images/doc/logo-light.svg b/website/src/images/doc/logo-light.svg new file mode 100644 index 0000000000..a65065c2ce --- /dev/null +++ b/website/src/images/doc/logo-light.svg @@ -0,0 +1,200 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/images/doc/logo2d.svg b/website/src/images/doc/logo2d.svg new file mode 100644 index 0000000000..b13fe5cdee --- /dev/null +++ b/website/src/images/doc/logo2d.svg @@ -0,0 +1,158 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/images/doc/logo2l.svg b/website/src/images/doc/logo2l.svg new file mode 100644 index 0000000000..9aef337c0c --- /dev/null +++ b/website/src/images/doc/logo2l.svg @@ -0,0 +1,216 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/images/doc/pr/pr1.png b/website/src/images/doc/pr/pr1.png new file mode 100644 index 0000000000..c0ea95842e Binary files /dev/null and b/website/src/images/doc/pr/pr1.png differ diff --git a/website/src/images/doc/pr/pr2.png b/website/src/images/doc/pr/pr2.png new file mode 100644 index 0000000000..6d5e6da998 Binary files /dev/null and b/website/src/images/doc/pr/pr2.png differ diff --git a/website/src/images/doc/pr/pr3.png b/website/src/images/doc/pr/pr3.png new file mode 100644 index 0000000000..77ddf62066 Binary files /dev/null and b/website/src/images/doc/pr/pr3.png differ diff --git a/website/src/images/doc/pr/pr4.png b/website/src/images/doc/pr/pr4.png new file mode 100644 index 0000000000..ed72f84764 Binary files /dev/null and b/website/src/images/doc/pr/pr4.png differ diff --git a/website/src/images/doc/pr/pr5.png b/website/src/images/doc/pr/pr5.png new file mode 100644 index 0000000000..cdb474e6fa Binary files /dev/null and b/website/src/images/doc/pr/pr5.png differ diff --git a/website/src/images/doc/pr/pr6.png b/website/src/images/doc/pr/pr6.png new file mode 100644 index 0000000000..3e1c8fa74a Binary files /dev/null and b/website/src/images/doc/pr/pr6.png differ diff --git a/website/src/images/doc/pr/pr7.png b/website/src/images/doc/pr/pr7.png new file mode 100644 index 0000000000..0acccaf25d Binary files /dev/null and b/website/src/images/doc/pr/pr7.png differ diff --git a/website/src/images/doc/pr/pr8.png b/website/src/images/doc/pr/pr8.png new file mode 100644 index 0000000000..28941427d9 Binary files /dev/null and b/website/src/images/doc/pr/pr8.png differ diff --git a/website/src/images/doc/pr/pr9.png b/website/src/images/doc/pr/pr9.png new file mode 100644 index 0000000000..0d738203d3 Binary files /dev/null and b/website/src/images/doc/pr/pr9.png differ diff --git a/docs/images/quick_1.png b/website/src/images/doc/quickstart/quick_1.png similarity index 100% rename from docs/images/quick_1.png rename to website/src/images/doc/quickstart/quick_1.png diff --git a/docs/images/quick_2.png b/website/src/images/doc/quickstart/quick_2.png similarity index 100% rename from docs/images/quick_2.png rename to website/src/images/doc/quickstart/quick_2.png diff --git a/docs/images/quick_3.png b/website/src/images/doc/quickstart/quick_3.png similarity index 100% rename from docs/images/quick_3.png rename to website/src/images/doc/quickstart/quick_3.png diff --git a/docs/images/quick_4.png b/website/src/images/doc/quickstart/quick_4.png similarity index 100% rename from docs/images/quick_4.png rename to website/src/images/doc/quickstart/quick_4.png diff --git a/docs/images/quick_5.png b/website/src/images/doc/quickstart/quick_5.png similarity index 100% rename from docs/images/quick_5.png rename to website/src/images/doc/quickstart/quick_5.png diff --git a/docs/images/quick_6.png b/website/src/images/doc/quickstart/quick_6.png similarity index 100% rename from docs/images/quick_6.png rename to website/src/images/doc/quickstart/quick_6.png diff --git a/docs/images/quick_7.png b/website/src/images/doc/quickstart/quick_7.png similarity index 100% rename from docs/images/quick_7.png rename to website/src/images/doc/quickstart/quick_7.png diff --git a/docs/images/quick_8.png b/website/src/images/doc/quickstart/quick_8.png similarity index 100% rename from docs/images/quick_8.png rename to website/src/images/doc/quickstart/quick_8.png diff --git a/website/src/images/doc/restore/restore1.png b/website/src/images/doc/restore/restore1.png new file mode 100644 index 0000000000..d9742f8e36 Binary files /dev/null and b/website/src/images/doc/restore/restore1.png differ diff --git a/website/src/images/doc/restore/restore2.png b/website/src/images/doc/restore/restore2.png new file mode 100644 index 0000000000..231257fa28 Binary files /dev/null and b/website/src/images/doc/restore/restore2.png differ diff --git a/website/src/images/doc/tutorial/docsVersionDropdown.png b/website/src/images/doc/tutorial/docsVersionDropdown.png new file mode 100644 index 0000000000..ff1cbe6889 Binary files /dev/null and b/website/src/images/doc/tutorial/docsVersionDropdown.png differ diff --git a/website/src/images/doc/tutorial/localeDropdown.png b/website/src/images/doc/tutorial/localeDropdown.png new file mode 100644 index 0000000000..d7163f9675 Binary files /dev/null and b/website/src/images/doc/tutorial/localeDropdown.png differ diff --git a/website/src/images/logo-dark.svg b/website/src/images/logo-dark.svg new file mode 100644 index 0000000000..c0625d0aea --- /dev/null +++ b/website/src/images/logo-dark.svg @@ -0,0 +1,161 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/src/pages/404.js b/website/src/pages/404.js new file mode 100644 index 0000000000..b98b1ba58f --- /dev/null +++ b/website/src/pages/404.js @@ -0,0 +1,14 @@ +import * as React from "react" + +import Layout from "../components/layout" +import Seo from "../components/seo" + +const NotFoundPage = () => ( + + +

404: Not Found

+

You just hit a route that doesn't exist... the sadness.

+
+) + +export default NotFoundPage diff --git a/website/src/pages/blog.scss b/website/src/pages/blog.scss new file mode 100644 index 0000000000..139b5480df --- /dev/null +++ b/website/src/pages/blog.scss @@ -0,0 +1,4 @@ +.blog-container { + height: 100vh; + background-color: #fff; +} diff --git a/website/src/pages/blogs.js b/website/src/pages/blogs.js new file mode 100644 index 0000000000..7d532a6214 --- /dev/null +++ b/website/src/pages/blogs.js @@ -0,0 +1,15 @@ +import * as React from "react" +import "./blog.scss" +// import AppHeaderWhite from "../components/AppHeaderWhite" +import AppFooter from "../components/AppFooter" +const Blogs = () => { + return ( +
+ {/* */} +
开发中
+ +
+ ) +} + +export default Blogs diff --git a/website/src/pages/documents/index.jsx b/website/src/pages/documents/index.jsx new file mode 100644 index 0000000000..556d81e4d5 --- /dev/null +++ b/website/src/pages/documents/index.jsx @@ -0,0 +1,48 @@ +import React from "react" +import { graphql, navigate } from "gatsby" +import { Skeleton } from "@mantine/core" + +import { buildMenu, getFileArr } from "../../util" +const IndexPage = props => { + const menuData = buildMenu(props.data.allFile.edges.map(item => item.node)) + const fileList = getFileArr(menuData.children) + if (fileList[0]) navigate(`/documents/${fileList.find(page => page.name === "快速开始").data.id}`) + return +} + +export const query = graphql` + query ($id: String) { + markdownRemark(id: { eq: $id }) { + html + id + parent { + ... on File { + id + name + modifiedTime + ino + } + } + } + allFile(filter: { sourceInstanceName: { eq: "docs" }, extension: { eq: "md" }, ctime: {} }) { + edges { + node { + id + name + parent { + id + ... on Directory { + id + name + } + } + relativePath + ctime + modifiedTime + } + } + } + } +` + +export default IndexPage diff --git a/website/src/pages/documents/index.scss b/website/src/pages/documents/index.scss new file mode 100644 index 0000000000..397aca7d37 --- /dev/null +++ b/website/src/pages/documents/index.scss @@ -0,0 +1,138 @@ +.container { + & span.gatsby-resp-image-background-image { + .gatsby-resp-image-image { + max-width: 90%; + margin: auto; + display: block; + } + } + + & ul { + list-style: none; + & li { + list-style-type: none; + display: block; + margin-bottom: 4px; + } + } + a { + color: #0984e3; + max-width: 100vw; + } + h1 { + display: block; + font-size: 2em; + margin-block-start: 0.67em; + margin-block-end: 0.67em; + margin-inline-start: 0px; + margin-inline-end: 0px; + &:first-child { + margin: 0 0 20px 0; + } + } + + h2 { + font-size: 25px; + } + + h3 { + display: block; + font-size: 1.17em; + margin-block-start: 1em; + margin-block-end: 1em; + margin-inline-start: 0px; + margin-inline-end: 0px; + } + + h4 { + font-size: 16px; + margin-block-start: 1.33em; + margin-block-end: 1.33em; + margin-inline-start: 0px; + margin-inline-end: 0px; + } + + h1, + h2, + h3, + h4, + h5 { + text-transform: capitalize; + font-weight: 550; + } + + p { + display: block; + margin-block-start: 1em; + margin-block-end: 1em; + margin-inline-start: 0px; + margin-inline-end: 0px; + overflow-wrap: break-word; + } + + pre { + background-color: #eee; + padding: 10px; + margin: 20px 0; + overflow: auto; + } + + table { + margin: 20px 0; + border-collapse: collapse; + display: block; + & > tr { + vertical-align: middle; + } + } + + td, + th { + border: 0.1px solid #535c68; + padding: 12px; + & a { + display: block; + } + &:nth-child(odd) { + background-color: #dfe6e9; + } + } + td { + display: table-cell; + vertical-align: inherit; + } + + ul { + display: block; + margin-block-start: 1em; + margin-block-end: 1em; + margin-inline-start: 0px; + margin-inline-end: 0px; + padding-inline-start: 40px; + } + li { + list-style-type: disc; + } + + blockquote p { + display: block; + margin-block-start: 1em; + margin-block-end: 1em; + margin-inline-start: 0px; + margin-inline-end: 0px; + padding: 0 20px; + background-color: rgb(243, 244, 244); + color: #999; + padding: 5px 10px; + position: relative; + &::before { + content: ""; + position: absolute; + top: 0; + left: 0; + height: 100%; + width: 2px; + background-color: #999; + } + } +} diff --git a/website/src/pages/documents/{MarkdownRemark.parent__(File)__id}.jsx b/website/src/pages/documents/{MarkdownRemark.parent__(File)__id}.jsx new file mode 100644 index 0000000000..feff82e29b --- /dev/null +++ b/website/src/pages/documents/{MarkdownRemark.parent__(File)__id}.jsx @@ -0,0 +1,114 @@ +import React from "react" +import { graphql, navigate } from "gatsby" +import { buildMenu, getFileArr } from "../../util" +import { Left, Right } from "@icon-park/react" +import "./index.scss" + +const BlogPost = props => { + const menuData = buildMenu(props.data.allFile.edges.map(item => item.node)) + const fileList = getFileArr(menuData.children) + const html = props.data.markdownRemark.html + const tableOfContents = props.data.markdownRemark.tableOfContents + + const location = window.location.pathname.split("/").pop() + const fileIndex = fileList.map(item => item.data.id).indexOf(location) + + const [preName, setPre] = React.useState("(无)") + const [nextName, setNext] = React.useState("(无)") + + React.useEffect(() => { + if (fileIndex > 0) { + setPre(fileList[fileIndex - 1].name) + } + if (fileIndex !== fileList.length - 1) { + setNext(fileList[fileIndex + 1].name) + } + }, []) + + function goPre() { + if (fileIndex === 0) return + + const target = fileList[fileIndex - 1] + navigate(`/documents/${target.data.id}`, { + state: { + fileIndex: fileIndex - 1, + fileList: fileList, + }, + }) + } + function goNext() { + if (fileIndex + 1 === fileList.length) return + const target = fileList[fileIndex + 1] + navigate(`/documents/${target.data.id}`, { + state: { + fileIndex: fileIndex + 1, + fileList: fileList, + }, + }) + } + + return ( +
+
+
+ +
+
+ + +
+
+ ) +} + +export const query = graphql` + query ($id: String) { + markdownRemark(id: { eq: $id }) { + tableOfContents(maxDepth: 2) + html + id + parent { + ... on File { + id + name + modifiedTime + ino + } + } + } + allFile(filter: { sourceInstanceName: { eq: "docs" }, extension: { eq: "md" }, ctime: {} }) { + edges { + node { + id + name + parent { + id + ... on Directory { + id + name + } + } + relativePath + ctime + modifiedTime + } + } + } + } +` + +export default BlogPost diff --git a/website/src/pages/download.js b/website/src/pages/download.js new file mode 100644 index 0000000000..48a88f19b4 --- /dev/null +++ b/website/src/pages/download.js @@ -0,0 +1,15 @@ +import * as React from "react" +import "./blog.scss" +// import AppHeaderWhite from "../components/AppHeaderWhite" +import AppFooter from "../components/AppFooter" +const DlownLoad = () => { + return ( +
+ {/* */} +
开发中
+ +
+ ) +} + +export default DlownLoad diff --git a/website/src/pages/examples/json/index.jsx b/website/src/pages/examples/json/index.jsx new file mode 100644 index 0000000000..d19903f68b --- /dev/null +++ b/website/src/pages/examples/json/index.jsx @@ -0,0 +1,40 @@ +import React from "react" +import { graphql, navigate } from "gatsby" +import { buildMenu, getFileArr } from "../../../util" +import { Skeleton } from "@mantine/core" + +const IndexPage = props => { + const menuData = buildMenu(props.data.allFile.edges.map(item => item.node)) + const fileList = getFileArr(menuData.children) + if (fileList[0]) navigate(`/examples/json/${fileList[0].data.id}`) + return +} + +export const query = graphql` + query ($id: String) { + jsonContent(id: { eq: $id }) { + id + content + } + allFile(filter: { sourceInstanceName: { eq: "examples" }, extension: { eq: "json" }, ctime: {} }) { + edges { + node { + id + name + parent { + id + ... on Directory { + id + name + } + } + relativePath + ctime + modifiedTime + } + } + } + } +` + +export default IndexPage diff --git a/website/src/pages/examples/json/{jsonContent.parent__(File)__id}.jsx b/website/src/pages/examples/json/{jsonContent.parent__(File)__id}.jsx new file mode 100644 index 0000000000..8b27f03ed9 --- /dev/null +++ b/website/src/pages/examples/json/{jsonContent.parent__(File)__id}.jsx @@ -0,0 +1,27 @@ +import React from "react" +import { graphql } from "gatsby" +import ReactJson from "react-json-view" + +const BlogPost = props => { + let json = {} + try { + json = JSON.parse(props.data.jsonContent.content) + } catch {} + + return ( +
+ +
+ ) +} + +export const query = graphql` + query ($id: String) { + jsonContent(id: { eq: $id }) { + id + content + } + } +` + +export default BlogPost diff --git a/website/src/pages/examples/sql/index.jsx b/website/src/pages/examples/sql/index.jsx new file mode 100644 index 0000000000..ea3f10c8dc --- /dev/null +++ b/website/src/pages/examples/sql/index.jsx @@ -0,0 +1,40 @@ +import React from "react" +import { graphql, navigate } from "gatsby" +import { buildMenu, getFileArr } from "../../../util" +import { Skeleton } from "@mantine/core" + +const IndexPage = props => { + const menuData = buildMenu(props.data.allFile.edges.map(item => item.node)) + const fileList = getFileArr(menuData.children) + if (fileList[0]) navigate(`/examples/sql/${fileList[0].data.id}`) + return +} + +export const query = graphql` + query ($id: String) { + jsonContent(id: { eq: $id }) { + id + content + } + allFile(filter: { sourceInstanceName: { eq: "examples" }, extension: { eq: "sql" }, ctime: {} }) { + edges { + node { + id + name + parent { + id + ... on Directory { + id + name + } + } + relativePath + ctime + modifiedTime + } + } + } + } +` + +export default IndexPage diff --git a/website/src/pages/examples/sql/index.scss b/website/src/pages/examples/sql/index.scss new file mode 100644 index 0000000000..01e4232779 --- /dev/null +++ b/website/src/pages/examples/sql/index.scss @@ -0,0 +1,32 @@ +.hljs-keyword { + color: rgb(193, 120, 218); +} + +.hljs-type { + color: rgb(193, 120, 218); +} + +.hljs-string, +.hljs-comment { + color: rgb(143, 195, 118); +} + +.hljs-comment { + word-break: break-all; + overflow: hidden; + width: 100%; + display: inline-block; + white-space: normal; +} + +.sql { + .pre { + display: flex; + justify-content: center; + width: 100%; + .code { + width: 100%; + background-color: red; + } + } +} diff --git a/website/src/pages/examples/sql/{jsonContent.parent__(File)__id}.jsx b/website/src/pages/examples/sql/{jsonContent.parent__(File)__id}.jsx new file mode 100644 index 0000000000..0ff18d1896 --- /dev/null +++ b/website/src/pages/examples/sql/{jsonContent.parent__(File)__id}.jsx @@ -0,0 +1,30 @@ +import React from "react" +import { graphql } from "gatsby" +import Highlight from "react-highlight" +import "./index.scss" + +const BlogPost = props => { + let sql = "" + try { + sql = props.data.jsonContent.content + } catch {} + + return ( +
+ + {sql} + +
+ ) +} + +export const query = graphql` + query ($id: String) { + jsonContent(id: { eq: $id }) { + id + content + } + } +` + +export default BlogPost diff --git a/website/src/pages/index.js b/website/src/pages/index.js new file mode 100644 index 0000000000..e05ca8d667 --- /dev/null +++ b/website/src/pages/index.js @@ -0,0 +1,21 @@ +import * as React from "react" +import AppHeader from "../components/AppHeader" +import AppBanner from "../components/AppBannner" +import AppFooter from "../components/AppFooter" +import AppCards from "../components/AppCards" +import AppMedium from "../components/AppMedium" +// import AppShowcase from "../components/AppShowcase" +import Seo from "../components/seo" +import "aos/dist/aos.css" +const IndexPage = () => ( + <> + + + + + + + +) + +export default IndexPage diff --git a/website/src/pages/page-2.js b/website/src/pages/page-2.js new file mode 100644 index 0000000000..74b96e08c0 --- /dev/null +++ b/website/src/pages/page-2.js @@ -0,0 +1,16 @@ +import * as React from "react" +import { Link } from "gatsby" + +import Layout from "../components/layout" +import Seo from "../components/seo" + +const SecondPage = () => ( + + +

Hi from the second page

+

Welcome to page 2

+ Go back to the homepage +
+) + +export default SecondPage diff --git a/website/src/pages/roadmap.js b/website/src/pages/roadmap.js new file mode 100644 index 0000000000..b1abd7c922 --- /dev/null +++ b/website/src/pages/roadmap.js @@ -0,0 +1,15 @@ +import * as React from "react" +import "./blog.scss" +// import AppHeaderWhite from "../components/AppHeaderWhite" +import AppFooter from "../components/AppFooter" +const RoadMap = () => { + return ( +
+ {/* */} +
开发中
+ +
+ ) +} + +export default RoadMap diff --git a/website/src/pages/using-ssr.js b/website/src/pages/using-ssr.js new file mode 100644 index 0000000000..6bbcff16ee --- /dev/null +++ b/website/src/pages/using-ssr.js @@ -0,0 +1,54 @@ +import * as React from "react" +import { Link } from "gatsby" + +import Layout from "../components/layout" +import Seo from "../components/seo" + +const UsingSSR = ({ serverData }) => { + return ( + + +

+ This page is rendered server-side +

+

+ This page is rendered server side every time the page is requested. + Reload it to see a(nother) random photo from{" "} + dog.ceo/api/breed/shiba/images/random: +

+ A random dog +

+ To learn more, head over to our{" "} + + documentation about Server Side Rendering + + . +

+ Go back to the homepage +
+ ) +} + +export default UsingSSR + +export async function getServerData() { + try { + const res = await fetch(`https://dog.ceo/api/breed/shiba/images/random`) + if (!res.ok) { + throw new Error(`Response failed`) + } + return { + props: await res.json(), + } + } catch (error) { + return { + status: 500, + headers: {}, + props: {}, + } + } +} diff --git a/website/src/pages/using-typescript.tsx b/website/src/pages/using-typescript.tsx new file mode 100644 index 0000000000..8018c1c905 --- /dev/null +++ b/website/src/pages/using-typescript.tsx @@ -0,0 +1,55 @@ +// If you don't want to use TypeScript you can delete this file! +import * as React from "react" +import { PageProps, Link, graphql } from "gatsby" + +import Layout from "../components/layout" +import Seo from "../components/seo" + +type DataProps = { + site: { + buildTime: string + } +} + +const UsingTypescript: React.FC> = ({ + data, + location, +}) => ( + + +

+ Gatsby supports TypeScript by default +

+

+ This means that you can create and write .ts/.tsx files for + your pages, components, and gatsby-* configuration files (for + example gatsby-config.ts). +

+

+ For type checking you'll want to install typescript via npm + and run tsc --init to create a tsconfig file. +

+

+ You're currently on the page {location.pathname} which was + built on {data.site.buildTime}. +

+

+ To learn more, head over to our{" "} + + documentation about TypeScript + + . +

+ Go back to the homepage +
+) + +export default UsingTypescript + +export const query = graphql` + { + site { + buildTime(formatString: "YYYY-MM-DD hh:mm a z") + } + } +` diff --git a/website/src/styles/global.css b/website/src/styles/global.css new file mode 100644 index 0000000000..cae8e9e280 --- /dev/null +++ b/website/src/styles/global.css @@ -0,0 +1,155 @@ +@tailwind base; +@tailwind components; +@tailwind utilities; + +@layer utilities { + .header-link[aria-current="page"]::before { + content: ""; + position: absolute; + width: 100%; + height: 100%; + left: 0; + top: 0; + background-color: #fff; + opacity: 0.1; + font-size: 15px; + } + .header-link[aria-current="page"]::after { + content: ""; + position: absolute; + left: 0; + bottom: 0; + height: 1px; + width: 100%; + background-color: #fff; + } + .hero { + height: calc(100vh - 90px); + } + @media screen and (max-width: 1504px) { + .hero { + background-position: 70% 50%; + } + } + @media screen and (max-width: 676px) { + .hero { + background-position: 65% 50%; + } + } + + .btn { + /* display: inline-block; */ + outline: none; + position: relative; + padding: 12px 45px; + border-radius: 100px; + text-transform: capitalize; + border: 0; + cursor: pointer; + transition: all 0.3s ease; + @media screen and (min-width: 1921px) { + padding: 24px 90px; + } + } + + .btn::after { + content: ""; + position: absolute; + z-index: -1; + border-radius: 100px; + width: 100%; + height: 100%; + left: 0; + top: 0; + transition: all 0.3s ease; + background-color: inherit; + background-image: inherit; + } + + .btn:hover { + transform: translateY(-3px); + } + + .btn:hover::after { + transform: scale(1.4, 1.6); + opacity: 0; + } + + .btn:active { + transform: translate(1px); + } + + .btn__large { + padding: 14px 35px; + @media screen and (max-width: 1504px) { + padding: 12px 20px; + font-size: 15px; + } + } + + .btn__blue { + background-image: linear-gradient(270deg, #057cea 0%, #00c5a0 100%); + color: #fff; + } + + .btn__white { + background-color: rgba(255, 255, 255, 0.1); + color: #fff; + } + + .btn__black { + background-color: #333; + color: #fff; + } + + p { + overflow-wrap: break-word !important; + } + .no-scrollbar::-webkit-scrollbar { + display: none; + } + + .no-scrollbar { + -ms-overflow-style: none; + scrollbar-width: none; + } + + .text-vsm { + font-size: 1vmax; + } + + .text-vbase { + font-size: 1.2vmax; + } + + .text-vlg { + font-size: 1.6vmax; + } + + .text-vxl { + font-size: 1.8vmax; + } + .text-v2xl { + font-size: 2vmax; + } +} + +@layer components { + .flex-padding { + @apply lg:px-[180px] md:px-[60px] px-[20px]; + } + .footer-link { + @apply text-gray-500 inline-block text-base hover:text-indigo-600; + } + .active { + @apply bg-gray-50 text-indigo-600 font-medium rounded-sm; + } + + .card { + @apply lg:w-1/3 md:w-1/2 w-4/5; + } + + .card-text { + @apply text-center md:text-base text-sm; + } +} diff --git a/website/src/styles/img/bg.png b/website/src/styles/img/bg.png new file mode 100644 index 0000000000..d7c89987d3 Binary files /dev/null and b/website/src/styles/img/bg.png differ diff --git a/website/src/styles/img/bg2x.png b/website/src/styles/img/bg2x.png new file mode 100644 index 0000000000..c20b3a591f Binary files /dev/null and b/website/src/styles/img/bg2x.png differ diff --git a/website/src/styles/img/block.png b/website/src/styles/img/block.png new file mode 100644 index 0000000000..2c2c75c04c Binary files /dev/null and b/website/src/styles/img/block.png differ diff --git a/website/src/styles/img/block2.png b/website/src/styles/img/block2.png new file mode 100644 index 0000000000..a29b86b992 Binary files /dev/null and b/website/src/styles/img/block2.png differ diff --git a/website/src/styles/img/block2@2x.png b/website/src/styles/img/block2@2x.png new file mode 100644 index 0000000000..873fd4aa06 Binary files /dev/null and b/website/src/styles/img/block2@2x.png differ diff --git a/website/src/styles/img/block2x.png b/website/src/styles/img/block2x.png new file mode 100644 index 0000000000..a330b4de86 Binary files /dev/null and b/website/src/styles/img/block2x.png differ diff --git a/website/src/templates/using-dsg.js b/website/src/templates/using-dsg.js new file mode 100644 index 0000000000..f2306afec4 --- /dev/null +++ b/website/src/templates/using-dsg.js @@ -0,0 +1,25 @@ +import * as React from "react" +import { Link } from "gatsby" + +import Layout from "../components/layout" +import Seo from "../components/seo" + +const UsingDSG = () => ( + + +

+ Hello from a DSG Page +

+

This page is not created until requested by a user.

+

+ To learn more, head over to our{" "} + + documentation about Deferred Static Generation + + . +

+ Go back to the homepage +
+) + +export default UsingDSG diff --git a/website/src/util/index.js b/website/src/util/index.js new file mode 100644 index 0000000000..37e87693da --- /dev/null +++ b/website/src/util/index.js @@ -0,0 +1,102 @@ +export const headerList = [ + { + name: "首页", + path: "/", + }, + { + name: "文档", + path: "/documents", + key: "documents", + }, + { + name: "案例", + path: [ + { + name: "sql", + link: "/examples/sql", + }, + { + name: "json", + link: "/examples/json", + }, + ], + key: "examples", + }, + // { + // name: "博客", + // path: "/blogs", + // key: "blogs", + // }, + { + name: "下载", + path: null, + url: "https://github.com/DTStack/chunjun/releases", + key: "download", + }, + // { + // name: "Roadmap", + // path: "/roadmap", + // key: "roadmap", + // }, +] + +export function buildMenu(nodes) { + let id = 1 + const root = { children: [] } + function linkToRoot(structrue, node) { + let rootRef = root + + for (let i = 0; i < structrue.length - 1; i++) { + let dirname = structrue[i] + let nextRef = rootRef.children.find(item => item.name === dirname) + if (!nextRef) { + nextRef = { + type: "dir", + name: dirname, + id: id++, + children: [], + parent: rootRef, + } + rootRef.children.push(nextRef) + } + rootRef = nextRef + } + rootRef.children.push({ + type: "file", + name: node.name, + data: node, + parent: rootRef, + }) + } + for (let i = 0; i < nodes.length; i++) { + let node = nodes[i] + let structrue = node.relativePath.split("/") + if (structrue.length === 1) { + root.children.push({ + type: "file", + name: node.name, + data: node, + parent: root, + }) + } else { + linkToRoot(structrue, node) + } + } + + return root +} + +export function getFileArr(list) { + const res = [] + travel(list) + function travel(list) { + list.forEach(item => { + if (item.type === "file") { + res.push(item) + } else { + travel(item.children || []) + } + }) + } + return res +} diff --git a/website/tailwind.config.js b/website/tailwind.config.js new file mode 100644 index 0000000000..1c07e1af2d --- /dev/null +++ b/website/tailwind.config.js @@ -0,0 +1,13 @@ +module.exports = { + content: ["./src/**/*.{jsx,ts,tsx}"], + theme: { + extend: { + backgroundImage: { + "hero-pattern": "url('./img/bg.png')", + block: "url('./img/block2x.png')", + block2: "url('./img/block2@2x.png')", + }, + }, + }, + plugins: [], +} diff --git a/website/yarn.lock b/website/yarn.lock new file mode 100644 index 0000000000..7e01b13fec --- /dev/null +++ b/website/yarn.lock @@ -0,0 +1,14126 @@ +# THIS IS AN AUTOGENERATED FILE. DO NOT EDIT THIS FILE DIRECTLY. +# yarn lockfile v1 + + +"@ampproject/remapping@^2.1.0": + version "2.2.0" + resolved "https://registry.npmmirror.com/@ampproject/remapping/-/remapping-2.2.0.tgz#56c133824780de3174aed5ab6834f3026790154d" + integrity sha512-qRmjj8nj9qmLTQXXmaR1cck3UXSRMPrbsLJAasZpF+t3riI71BXed5ebIOYwQntykeZuhjsdweEc9BxH5Jc26w== + dependencies: + "@jridgewell/gen-mapping" "^0.1.0" + "@jridgewell/trace-mapping" "^0.3.9" + +"@ant-design/colors@^6.0.0": + version "6.0.0" + resolved "https://registry.npmmirror.com/@ant-design/colors/-/colors-6.0.0.tgz#9b9366257cffcc47db42b9d0203bb592c13c0298" + integrity sha512-qAZRvPzfdWHtfameEGP2Qvuf838NhergR35o+EuVyB5XvSA98xod5r4utvi4TJ3ywmevm290g9nsCG5MryrdWQ== + dependencies: + "@ctrl/tinycolor" "^3.4.0" + +"@ant-design/icons-svg@^4.2.1": + version "4.2.1" + resolved "https://registry.npmmirror.com/@ant-design/icons-svg/-/icons-svg-4.2.1.tgz#8630da8eb4471a4aabdaed7d1ff6a97dcb2cf05a" + integrity sha512-EB0iwlKDGpG93hW8f85CTJTs4SvMX7tt5ceupvhALp1IF44SeUFOMhKUOYqpsoYWQKAOuTRDMqn75rEaKDp0Xw== + +"@ant-design/icons@^4.7.0": + version "4.7.0" + resolved "https://registry.npmmirror.com/@ant-design/icons/-/icons-4.7.0.tgz#8c3cbe0a556ba92af5dc7d1e70c0b25b5179af0f" + integrity sha512-aoB4Z7JA431rt6d4u+8xcNPPCrdufSRMUOpxa1ab6mz1JCQZOEVolj2WVs/tDFmN62zzK30mNelEsprLYsSF3g== + dependencies: + "@ant-design/colors" "^6.0.0" + "@ant-design/icons-svg" "^4.2.1" + "@babel/runtime" "^7.11.2" + classnames "^2.2.6" + rc-util "^5.9.4" + +"@ant-design/react-slick@~0.29.1": + version "0.29.2" + resolved "https://registry.npmmirror.com/@ant-design/react-slick/-/react-slick-0.29.2.tgz#53e6a7920ea3562eebb304c15a7fc2d7e619d29c" + integrity sha512-kgjtKmkGHa19FW21lHnAfyyH9AAoh35pBdcJ53rHmQ3O+cfFHGHnUbj/HFrRNJ5vIts09FKJVAD8RpaC+RaWfA== + dependencies: + "@babel/runtime" "^7.10.4" + classnames "^2.2.5" + json2mq "^0.2.0" + lodash "^4.17.21" + resize-observer-polyfill "^1.5.1" + +"@ardatan/aggregate-error@0.0.6": + version "0.0.6" + resolved "https://registry.npmmirror.com/@ardatan/aggregate-error/-/aggregate-error-0.0.6.tgz#fe6924771ea40fc98dc7a7045c2e872dc8527609" + integrity sha512-vyrkEHG1jrukmzTPtyWB4NLPauUw5bQeg4uhn8f+1SSynmrOcyvlb1GKQjjgoBzElLdfXCRYX8UnBlhklOHYRQ== + dependencies: + tslib "~2.0.1" + +"@ardatan/relay-compiler@12.0.0": + version "12.0.0" + resolved "https://registry.npmmirror.com/@ardatan/relay-compiler/-/relay-compiler-12.0.0.tgz#2e4cca43088e807adc63450e8cab037020e91106" + integrity sha512-9anThAaj1dQr6IGmzBMcfzOQKTa5artjuPmw8NYK/fiGEMjADbSguBY2FMDykt+QhilR3wc9VA/3yVju7JHg7Q== + dependencies: + "@babel/core" "^7.14.0" + "@babel/generator" "^7.14.0" + "@babel/parser" "^7.14.0" + "@babel/runtime" "^7.0.0" + "@babel/traverse" "^7.14.0" + "@babel/types" "^7.0.0" + babel-preset-fbjs "^3.4.0" + chalk "^4.0.0" + fb-watchman "^2.0.0" + fbjs "^3.0.0" + glob "^7.1.1" + immutable "~3.7.6" + invariant "^2.2.4" + nullthrows "^1.1.1" + relay-runtime "12.0.0" + signedsource "^1.0.0" + yargs "^15.3.1" + +"@babel/code-frame@7.12.11": + version "7.12.11" + resolved "https://registry.npmmirror.com/@babel/code-frame/-/code-frame-7.12.11.tgz#f4ad435aa263db935b8f10f2c552d23fb716a63f" + integrity sha512-Zt1yodBx1UcyiePMSkWnU4hPqhwq7hGi2nFL1LeA3EUl+q2LQx16MISgJ0+z7dnmgvP9QtIleuETGOiOH1RcIw== + dependencies: + "@babel/highlight" "^7.10.4" + +"@babel/code-frame@^7.0.0", "@babel/code-frame@^7.14.0", "@babel/code-frame@^7.16.0", "@babel/code-frame@^7.16.7", "@babel/code-frame@^7.8.3": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/code-frame/-/code-frame-7.16.7.tgz#44416b6bd7624b998f5b1af5d470856c40138789" + integrity sha512-iAXqUn8IIeBTNd72xsFlgaXHkMBMt6y4HJp1tIaK465CWLT/fG1aqB7ykr95gHHmlBdGbFeWWfyB4NJJ0nmeIg== + dependencies: + "@babel/highlight" "^7.16.7" + +"@babel/compat-data@^7.13.11", "@babel/compat-data@^7.17.10": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/compat-data/-/compat-data-7.18.5.tgz#acac0c839e317038c73137fbb6ef71a1d6238471" + integrity sha512-BxhE40PVCBxVEJsSBhB6UWyAuqJRxGsAw8BdHMJ3AKGydcwuWW4kOO3HmqBQAdcq/OP+/DlTVxLvsCzRTnZuGg== + +"@babel/core@^7.14.0", "@babel/core@^7.15.5": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/core/-/core-7.18.5.tgz#c597fa680e58d571c28dda9827669c78cdd7f000" + integrity sha512-MGY8vg3DxMnctw0LdvSEojOsumc70g0t18gNyUdAZqB1Rpd1Bqo/svHGvt+UJ6JcGX+DIekGFDxxIWofBxLCnQ== + dependencies: + "@ampproject/remapping" "^2.1.0" + "@babel/code-frame" "^7.16.7" + "@babel/generator" "^7.18.2" + "@babel/helper-compilation-targets" "^7.18.2" + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helpers" "^7.18.2" + "@babel/parser" "^7.18.5" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.5" + "@babel/types" "^7.18.4" + convert-source-map "^1.7.0" + debug "^4.1.0" + gensync "^1.0.0-beta.2" + json5 "^2.2.1" + semver "^6.3.0" + +"@babel/eslint-parser@^7.15.4": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/eslint-parser/-/eslint-parser-7.18.2.tgz#e14dee36c010edfb0153cf900c2b0815e82e3245" + integrity sha512-oFQYkE8SuH14+uR51JVAmdqwKYXGRjEXx7s+WiagVjqQ+HPE+nnwyF2qlVG8evUsUHmPcA+6YXMEDbIhEyQc5A== + dependencies: + eslint-scope "^5.1.1" + eslint-visitor-keys "^2.1.0" + semver "^6.3.0" + +"@babel/generator@^7.14.0", "@babel/generator@^7.16.8", "@babel/generator@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/generator/-/generator-7.18.2.tgz#33873d6f89b21efe2da63fe554460f3df1c5880d" + integrity sha512-W1lG5vUwFvfMd8HVXqdfbuG7RuaSrTCCD8cl8fP8wOivdbtbIg2Db3IWUcgvfxKbbn6ZBGYRW/Zk1MIwK49mgw== + dependencies: + "@babel/types" "^7.18.2" + "@jridgewell/gen-mapping" "^0.3.0" + jsesc "^2.5.1" + +"@babel/helper-annotate-as-pure@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.16.7.tgz#bb2339a7534a9c128e3102024c60760a3a7f3862" + integrity sha512-s6t2w/IPQVTAET1HitoowRGXooX8mCgtuP5195wD/QJPV6wYjpujCGF7JuMODVX2ZAJOf1GT6DT9MHEZvLOFSw== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-builder-binary-assignment-operator-visitor@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.16.7.tgz#38d138561ea207f0f69eb1626a418e4f7e6a580b" + integrity sha512-C6FdbRaxYjwVu/geKW4ZeQ0Q31AftgRcdSnZ5/jsH6BzCJbtvXvhpfkbkThYSuutZA7nCXpPR6AD9zd1dprMkA== + dependencies: + "@babel/helper-explode-assignable-expression" "^7.16.7" + "@babel/types" "^7.16.7" + +"@babel/helper-compilation-targets@^7.13.0", "@babel/helper-compilation-targets@^7.16.7", "@babel/helper-compilation-targets@^7.17.10", "@babel/helper-compilation-targets@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.18.2.tgz#67a85a10cbd5fc7f1457fec2e7f45441dc6c754b" + integrity sha512-s1jnPotJS9uQnzFtiZVBUxe67CuBa679oWFHpxYYnTpRL/1ffhyX44R9uYiXoa/pLXcY9H2moJta0iaanlk/rQ== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-validator-option" "^7.16.7" + browserslist "^4.20.2" + semver "^6.3.0" + +"@babel/helper-create-class-features-plugin@^7.17.12", "@babel/helper-create-class-features-plugin@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.18.0.tgz#fac430912606331cb075ea8d82f9a4c145a4da19" + integrity sha512-Kh8zTGR9de3J63e5nS0rQUdRs/kbtwoeQQ0sriS0lItjC96u8XXZN6lKpuyWd2coKSU13py/y+LTmThLuVX0Pg== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-environment-visitor" "^7.16.7" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-member-expression-to-functions" "^7.17.7" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/helper-replace-supers" "^7.16.7" + "@babel/helper-split-export-declaration" "^7.16.7" + +"@babel/helper-create-regexp-features-plugin@^7.16.7", "@babel/helper-create-regexp-features-plugin@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.17.12.tgz#bb37ca467f9694bbe55b884ae7a5cc1e0084e4fd" + integrity sha512-b2aZrV4zvutr9AIa6/gA3wsZKRwTKYoDxYiFKcESS3Ug2GTXzwBEvMuuFLhCQpEnRXs1zng4ISAXSUxxKBIcxw== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + regexpu-core "^5.0.1" + +"@babel/helper-define-polyfill-provider@^0.3.1": + version "0.3.1" + resolved "https://registry.npmmirror.com/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.3.1.tgz#52411b445bdb2e676869e5a74960d2d3826d2665" + integrity sha512-J9hGMpJQmtWmj46B3kBHmL38UhJGhYX7eqkcq+2gsstyYt341HmPeWspihX43yVRA0mS+8GGk2Gckc7bY/HCmA== + dependencies: + "@babel/helper-compilation-targets" "^7.13.0" + "@babel/helper-module-imports" "^7.12.13" + "@babel/helper-plugin-utils" "^7.13.0" + "@babel/traverse" "^7.13.0" + debug "^4.1.1" + lodash.debounce "^4.0.8" + resolve "^1.14.2" + semver "^6.1.2" + +"@babel/helper-environment-visitor@^7.16.7", "@babel/helper-environment-visitor@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.2.tgz#8a6d2dedb53f6bf248e31b4baf38739ee4a637bd" + integrity sha512-14GQKWkX9oJzPiQQ7/J36FTXcD4kSp8egKjO9nINlSKiHITRA9q/R74qu8S9xlc/b/yjsJItQUeeh3xnGN0voQ== + +"@babel/helper-explode-assignable-expression@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-explode-assignable-expression/-/helper-explode-assignable-expression-7.16.7.tgz#12a6d8522fdd834f194e868af6354e8650242b7a" + integrity sha512-KyUenhWMC8VrxzkGP0Jizjo4/Zx+1nNZhgocs+gLzyZyB8SHidhoq9KK/8Ato4anhwsivfkBLftky7gvzbZMtQ== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-function-name@^7.16.7", "@babel/helper-function-name@^7.17.9": + version "7.17.9" + resolved "https://registry.npmmirror.com/@babel/helper-function-name/-/helper-function-name-7.17.9.tgz#136fcd54bc1da82fcb47565cf16fd8e444b1ff12" + integrity sha512-7cRisGlVtiVqZ0MW0/yFB4atgpGLWEHUVYnb448hZK4x+vih0YO5UoS11XIYtZYqHd0dIPMdUSv8q5K4LdMnIg== + dependencies: + "@babel/template" "^7.16.7" + "@babel/types" "^7.17.0" + +"@babel/helper-hoist-variables@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.16.7.tgz#86bcb19a77a509c7b77d0e22323ef588fa58c246" + integrity sha512-m04d/0Op34H5v7pbZw6pSKP7weA6lsMvfiIAMeIvkY/R4xQtBSMFEigu9QTZ2qB/9l22vsxtM8a+Q8CzD255fg== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-member-expression-to-functions@^7.17.7": + version "7.17.7" + resolved "https://registry.npmmirror.com/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.17.7.tgz#a34013b57d8542a8c4ff8ba3f747c02452a4d8c4" + integrity sha512-thxXgnQ8qQ11W2wVUObIqDL4p148VMxkt5T/qpN5k2fboRyzFGFmKsTGViquyM5QHKUy48OZoca8kw4ajaDPyw== + dependencies: + "@babel/types" "^7.17.0" + +"@babel/helper-module-imports@^7.0.0-beta.49", "@babel/helper-module-imports@^7.12.13", "@babel/helper-module-imports@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-module-imports/-/helper-module-imports-7.16.7.tgz#25612a8091a999704461c8a222d0efec5d091437" + integrity sha512-LVtS6TqjJHFc+nYeITRo6VLXve70xmq7wPhWTqDJusJEgGmkAACWwMiTNrvfoQo6hEhFwAIixNkvB0jPXDL8Wg== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-module-transforms@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/helper-module-transforms/-/helper-module-transforms-7.18.0.tgz#baf05dec7a5875fb9235bd34ca18bad4e21221cd" + integrity sha512-kclUYSUBIjlvnzN2++K9f2qzYKFgjmnmjwL4zlmU5f8ZtzgWe8s0rUPSTGy2HmK4P8T52MQsS+HTQAgZd3dMEA== + dependencies: + "@babel/helper-environment-visitor" "^7.16.7" + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-simple-access" "^7.17.7" + "@babel/helper-split-export-declaration" "^7.16.7" + "@babel/helper-validator-identifier" "^7.16.7" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.0" + "@babel/types" "^7.18.0" + +"@babel/helper-optimise-call-expression@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.16.7.tgz#a34e3560605abbd31a18546bd2aad3e6d9a174f2" + integrity sha512-EtgBhg7rd/JcnpZFXpBy0ze1YRfdm7BnBX4uKMBd3ixa3RGAE002JZB66FJyNH7g0F38U05pXmA5P8cBh7z+1w== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-plugin-utils@^7.0.0", "@babel/helper-plugin-utils@^7.10.4", "@babel/helper-plugin-utils@^7.12.13", "@babel/helper-plugin-utils@^7.13.0", "@babel/helper-plugin-utils@^7.14.5", "@babel/helper-plugin-utils@^7.16.7", "@babel/helper-plugin-utils@^7.17.12", "@babel/helper-plugin-utils@^7.8.0", "@babel/helper-plugin-utils@^7.8.3": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.17.12.tgz#86c2347da5acbf5583ba0a10aed4c9bf9da9cf96" + integrity sha512-JDkf04mqtN3y4iAbO1hv9U2ARpPyPL1zqyWs/2WG1pgSq9llHFjStX5jdxb84himgJm+8Ng+x0oiWF/nw/XQKA== + +"@babel/helper-remap-async-to-generator@^7.16.8": + version "7.16.8" + resolved "https://registry.npmmirror.com/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.16.8.tgz#29ffaade68a367e2ed09c90901986918d25e57e3" + integrity sha512-fm0gH7Flb8H51LqJHy3HJ3wnE1+qtYR2A99K06ahwrawLdOFsCEWjZOrYricXJHoPSudNKxrMBUPEIPxiIIvBw== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-wrap-function" "^7.16.8" + "@babel/types" "^7.16.8" + +"@babel/helper-replace-supers@^7.16.7", "@babel/helper-replace-supers@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/helper-replace-supers/-/helper-replace-supers-7.18.2.tgz#41fdfcc9abaf900e18ba6e5931816d9062a7b2e0" + integrity sha512-XzAIyxx+vFnrOxiQrToSUOzUOn0e1J2Li40ntddek1Y69AXUTXoDJ40/D5RdjFu7s7qHiaeoTiempZcbuVXh2Q== + dependencies: + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-member-expression-to-functions" "^7.17.7" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/traverse" "^7.18.2" + "@babel/types" "^7.18.2" + +"@babel/helper-simple-access@^7.17.7", "@babel/helper-simple-access@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/helper-simple-access/-/helper-simple-access-7.18.2.tgz#4dc473c2169ac3a1c9f4a51cfcd091d1c36fcff9" + integrity sha512-7LIrjYzndorDY88MycupkpQLKS1AFfsVRm2k/9PtKScSy5tZq0McZTj+DiMRynboZfIqOKvo03pmhTaUgiD6fQ== + dependencies: + "@babel/types" "^7.18.2" + +"@babel/helper-skip-transparent-expression-wrappers@^7.16.0": + version "7.16.0" + resolved "https://registry.npmmirror.com/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.16.0.tgz#0ee3388070147c3ae051e487eca3ebb0e2e8bb09" + integrity sha512-+il1gTy0oHwUsBQZyJvukbB4vPMdcYBrFHa0Uc4AizLxbq6BOYC51Rv4tWocX9BLBDLZ4kc6qUFpQ6HRgL+3zw== + dependencies: + "@babel/types" "^7.16.0" + +"@babel/helper-split-export-declaration@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.16.7.tgz#0b648c0c42da9d3920d85ad585f2778620b8726b" + integrity sha512-xbWoy/PFoxSWazIToT9Sif+jJTlrMcndIsaOKvTA6u7QEo7ilkRZpjew18/W3c7nm8fXdUDXh02VXTbZ0pGDNw== + dependencies: + "@babel/types" "^7.16.7" + +"@babel/helper-validator-identifier@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.16.7.tgz#e8c602438c4a8195751243da9031d1607d247cad" + integrity sha512-hsEnFemeiW4D08A5gUAZxLBTXpZ39P+a+DGDsHw1yxqyQ/jzFEnxf5uTEGp+3bzAbNOxU1paTgYS4ECU/IgfDw== + +"@babel/helper-validator-option@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/helper-validator-option/-/helper-validator-option-7.16.7.tgz#b203ce62ce5fe153899b617c08957de860de4d23" + integrity sha512-TRtenOuRUVo9oIQGPC5G9DgK4743cdxvtOw0weQNpZXaS16SCBi5MNjZF8vba3ETURjZpTbVn7Vvcf2eAwFozQ== + +"@babel/helper-wrap-function@^7.16.8": + version "7.16.8" + resolved "https://registry.npmmirror.com/@babel/helper-wrap-function/-/helper-wrap-function-7.16.8.tgz#58afda087c4cd235de92f7ceedebca2c41274200" + integrity sha512-8RpyRVIAW1RcDDGTA+GpPAwV22wXCfKOoM9bet6TLkGIFTkRQSkH1nMQ5Yet4MpoXe1ZwHPVtNasc2w0uZMqnw== + dependencies: + "@babel/helper-function-name" "^7.16.7" + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.16.8" + "@babel/types" "^7.16.8" + +"@babel/helpers@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/helpers/-/helpers-7.18.2.tgz#970d74f0deadc3f5a938bfa250738eb4ac889384" + integrity sha512-j+d+u5xT5utcQSzrh9p+PaJX94h++KN+ng9b9WEJq7pkUPAd61FGqhjuUEdfknb3E/uDBb7ruwEeKkIxNJPIrg== + dependencies: + "@babel/template" "^7.16.7" + "@babel/traverse" "^7.18.2" + "@babel/types" "^7.18.2" + +"@babel/highlight@^7.10.4", "@babel/highlight@^7.16.7": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/highlight/-/highlight-7.17.12.tgz#257de56ee5afbd20451ac0a75686b6b404257351" + integrity sha512-7yykMVF3hfZY2jsHZEEgLc+3x4o1O+fYyULu11GynEUQNwB6lua+IIQn1FiJxNucd5UlyJryrwsOh8PL9Sn8Qg== + dependencies: + "@babel/helper-validator-identifier" "^7.16.7" + chalk "^2.0.0" + js-tokens "^4.0.0" + +"@babel/parser@^7.14.0", "@babel/parser@^7.15.5", "@babel/parser@^7.16.7", "@babel/parser@^7.16.8", "@babel/parser@^7.18.5": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/parser/-/parser-7.18.5.tgz#337062363436a893a2d22faa60be5bb37091c83c" + integrity sha512-YZWVaglMiplo7v8f1oMQ5ZPQr0vn7HPeZXxXWsxXJRjGVrzUFn9OxFQl1sb5wzfootjA/yChhW84BV+383FSOw== + +"@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.17.12.tgz#1dca338caaefca368639c9ffb095afbd4d420b1e" + integrity sha512-xCJQXl4EeQ3J9C4yOmpTrtVGmzpm2iSzyxbkZHw7UCnZBftHpF/hpII80uWVyVrc40ytIClHjgWGTG1g/yB+aw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.17.12.tgz#0d498ec8f0374b1e2eb54b9cb2c4c78714c77753" + integrity sha512-/vt0hpIw0x4b6BLKUkwlvEoiGZYYLNZ96CzyHYPbtG2jZGz6LBe7/V+drYrc/d+ovrF9NBi0pmtvmNb/FsWtRQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + "@babel/plugin-proposal-optional-chaining" "^7.17.12" + +"@babel/plugin-proposal-async-generator-functions@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-async-generator-functions/-/plugin-proposal-async-generator-functions-7.17.12.tgz#094a417e31ce7e692d84bab06c8e2a607cbeef03" + integrity sha512-RWVvqD1ooLKP6IqWTA5GyFVX2isGEgC5iFxKzfYOIy/QEFdxYyCybBDtIGjipHpb9bDWHzcqGqFakf+mVmBTdQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-remap-async-to-generator" "^7.16.8" + "@babel/plugin-syntax-async-generators" "^7.8.4" + +"@babel/plugin-proposal-class-properties@^7.0.0", "@babel/plugin-proposal-class-properties@^7.14.0", "@babel/plugin-proposal-class-properties@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-class-properties/-/plugin-proposal-class-properties-7.17.12.tgz#84f65c0cc247d46f40a6da99aadd6438315d80a4" + integrity sha512-U0mI9q8pW5Q9EaTHFPwSVusPMV/DV9Mm8p7csqROFLtIE9rBF5piLqyrBGigftALrBcsBGu4m38JneAe7ZDLXw== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-proposal-class-static-block@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-class-static-block/-/plugin-proposal-class-static-block-7.18.0.tgz#7d02253156e3c3793bdb9f2faac3a1c05f0ba710" + integrity sha512-t+8LsRMMDE74c6sV7KShIw13sqbqd58tlqNrsWoWBTIMw7SVQ0cZ905wLNS/FBCy/3PyooRHLFFlfrUNyyz5lA== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + +"@babel/plugin-proposal-dynamic-import@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-dynamic-import/-/plugin-proposal-dynamic-import-7.16.7.tgz#c19c897eaa46b27634a00fee9fb7d829158704b2" + integrity sha512-I8SW9Ho3/8DRSdmDdH3gORdyUuYnk1m4cMxUAdu5oy4n3OfN8flDEH+d60iG7dUfi0KkYwSvoalHzzdRzpWHTg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + +"@babel/plugin-proposal-export-namespace-from@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-export-namespace-from/-/plugin-proposal-export-namespace-from-7.17.12.tgz#b22864ccd662db9606edb2287ea5fd1709f05378" + integrity sha512-j7Ye5EWdwoXOpRmo5QmRyHPsDIe6+u70ZYZrd7uz+ebPYFKfRcLcNu3Ro0vOlJ5zuv8rU7xa+GttNiRzX56snQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + +"@babel/plugin-proposal-json-strings@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-json-strings/-/plugin-proposal-json-strings-7.17.12.tgz#f4642951792437233216d8c1af370bb0fbff4664" + integrity sha512-rKJ+rKBoXwLnIn7n6o6fulViHMrOThz99ybH+hKHcOZbnN14VuMnH9fo2eHE69C8pO4uX1Q7t2HYYIDmv8VYkg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-json-strings" "^7.8.3" + +"@babel/plugin-proposal-logical-assignment-operators@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-logical-assignment-operators/-/plugin-proposal-logical-assignment-operators-7.17.12.tgz#c64a1bcb2b0a6d0ed2ff674fd120f90ee4b88a23" + integrity sha512-EqFo2s1Z5yy+JeJu7SFfbIUtToJTVlC61/C7WLKDntSw4Sz6JNAIfL7zQ74VvirxpjB5kz/kIx0gCcb+5OEo2Q== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + +"@babel/plugin-proposal-nullish-coalescing-operator@^7.14.5", "@babel/plugin-proposal-nullish-coalescing-operator@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-nullish-coalescing-operator/-/plugin-proposal-nullish-coalescing-operator-7.17.12.tgz#1e93079bbc2cbc756f6db6a1925157c4a92b94be" + integrity sha512-ws/g3FSGVzv+VH86+QvgtuJL/kR67xaEIF2x0iPqdDfYW6ra6JF3lKVBkWynRLcNtIC1oCTfDRVxmm2mKzy+ag== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + +"@babel/plugin-proposal-numeric-separator@^7.14.5", "@babel/plugin-proposal-numeric-separator@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-numeric-separator/-/plugin-proposal-numeric-separator-7.16.7.tgz#d6b69f4af63fb38b6ca2558442a7fb191236eba9" + integrity sha512-vQgPMknOIgiuVqbokToyXbkY/OmmjAzr/0lhSIbG/KmnzXPGwW/AdhdKpi+O4X/VkWiWjnkKOBiqJrTaC98VKw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + +"@babel/plugin-proposal-object-rest-spread@^7.0.0", "@babel/plugin-proposal-object-rest-spread@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-object-rest-spread/-/plugin-proposal-object-rest-spread-7.18.0.tgz#79f2390c892ba2a68ec112eb0d895cfbd11155e8" + integrity sha512-nbTv371eTrFabDfHLElkn9oyf9VG+VKK6WMzhY2o4eHKaG19BToD9947zzGMO6I/Irstx9d8CwX6njPNIAR/yw== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-compilation-targets" "^7.17.10" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-transform-parameters" "^7.17.12" + +"@babel/plugin-proposal-optional-catch-binding@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-optional-catch-binding/-/plugin-proposal-optional-catch-binding-7.16.7.tgz#c623a430674ffc4ab732fd0a0ae7722b67cb74cf" + integrity sha512-eMOH/L4OvWSZAE1VkHbr1vckLG1WUcHGJSLqqQwl2GaUqG6QjddvrOaTUMNYiv77H5IKPMZ9U9P7EaHwvAShfA== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + +"@babel/plugin-proposal-optional-chaining@^7.14.5", "@babel/plugin-proposal-optional-chaining@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-optional-chaining/-/plugin-proposal-optional-chaining-7.17.12.tgz#f96949e9bacace3a9066323a5cf90cfb9de67174" + integrity sha512-7wigcOs/Z4YWlK7xxjkvaIw84vGhDv/P1dFGQap0nHkc8gFKY/r+hXc8Qzf5k1gY7CvGIcHqAnOagVKJJ1wVOQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + +"@babel/plugin-proposal-private-methods@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-private-methods/-/plugin-proposal-private-methods-7.17.12.tgz#c2ca3a80beb7539289938da005ad525a038a819c" + integrity sha512-SllXoxo19HmxhDWm3luPz+cPhtoTSKLJE9PXshsfrOzBqs60QP0r8OaJItrPhAj0d7mZMnNF0Y1UUggCDgMz1A== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-proposal-private-property-in-object@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.17.12.tgz#b02efb7f106d544667d91ae97405a9fd8c93952d" + integrity sha512-/6BtVi57CJfrtDNKfK5b66ydK2J5pXUKBKSPD2G1whamMuEnZWgoOIfO8Vf9F/DoD4izBLD/Au4NMQfruzzykg== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-create-class-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + +"@babel/plugin-proposal-unicode-property-regex@^7.17.12", "@babel/plugin-proposal-unicode-property-regex@^7.4.4": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-proposal-unicode-property-regex/-/plugin-proposal-unicode-property-regex-7.17.12.tgz#3dbd7a67bd7f94c8238b394da112d86aaf32ad4d" + integrity sha512-Wb9qLjXf3ZazqXA7IvI7ozqRIXIGPtSo+L5coFmEkhTQK18ao4UDDD0zdTGAarmbLj2urpRwrc6893cu5Bfh0A== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-async-generators@^7.8.4": + version "7.8.4" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz#a983fb1aeb2ec3f6ed042a210f640e90e786fe0d" + integrity sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-class-properties@^7.0.0", "@babel/plugin-syntax-class-properties@^7.12.13": + version "7.12.13" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz#b5c987274c4a3a82b89714796931a6b53544ae10" + integrity sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA== + dependencies: + "@babel/helper-plugin-utils" "^7.12.13" + +"@babel/plugin-syntax-class-static-block@^7.14.5": + version "7.14.5" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz#195df89b146b4b78b3bf897fd7a257c84659d406" + integrity sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-dynamic-import@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz#62bf98b2da3cd21d626154fc96ee5b3cb68eacb3" + integrity sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-export-namespace-from@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz#028964a9ba80dbc094c915c487ad7c4e7a66465a" + integrity sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.3" + +"@babel/plugin-syntax-flow@^7.0.0", "@babel/plugin-syntax-flow@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-flow/-/plugin-syntax-flow-7.17.12.tgz#23d852902acd19f42923fca9d0f196984d124e73" + integrity sha512-B8QIgBvkIG6G2jgsOHQUist7Sm0EBLDCx8sen072IwqNuzMegZNXrYnSv77cYzA8mLDZAfQYqsLIhimiP1s2HQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-import-assertions@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-import-assertions/-/plugin-syntax-import-assertions-7.17.12.tgz#58096a92b11b2e4e54b24c6a0cc0e5e607abcedd" + integrity sha512-n/loy2zkq9ZEM8tEOwON9wTQSTNDTDEz6NujPtJGLU7qObzT1N4c4YZZf8E6ATB2AjNQg/Ib2AIpO03EZaCehw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-json-strings@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz#01ca21b668cd8218c9e640cb6dd88c5412b2c96a" + integrity sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-jsx@^7.0.0", "@babel/plugin-syntax-jsx@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.17.12.tgz#834035b45061983a491f60096f61a2e7c5674a47" + integrity sha512-spyY3E3AURfxh/RHtjx5j6hs8am5NbUBGfcZ2vB3uShSpZdQyXSf5rR5Mk76vbtlAZOelyVQ71Fg0x9SG4fsog== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-syntax-logical-assignment-operators@^7.10.4": + version "7.10.4" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz#ca91ef46303530448b906652bac2e9fe9941f699" + integrity sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-nullish-coalescing-operator@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz#167ed70368886081f74b5c36c65a88c03b66d1a9" + integrity sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-numeric-separator@^7.10.4": + version "7.10.4" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz#b9b070b3e33570cd9fd07ba7fa91c0dd37b9af97" + integrity sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-object-rest-spread@^7.0.0", "@babel/plugin-syntax-object-rest-spread@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz#60e225edcbd98a640332a2e72dd3e66f1af55871" + integrity sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-catch-binding@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz#6111a265bcfb020eb9efd0fdfd7d26402b9ed6c1" + integrity sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-chaining@^7.8.3": + version "7.8.3" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz#4f69c2ab95167e0180cd5336613f8c5788f7d48a" + integrity sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-private-property-in-object@^7.14.5": + version "7.14.5" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz#0dc6671ec0ea22b6e94a1114f857970cd39de1ad" + integrity sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-top-level-await@^7.14.5": + version "7.14.5" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz#c1cfdadc35a646240001f06138247b741c34d94c" + integrity sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-typescript@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.17.12.tgz#b54fc3be6de734a56b87508f99d6428b5b605a7b" + integrity sha512-TYY0SXFiO31YXtNg3HtFwNJHjLsAyIIhAhNWkQ5whPPS7HWUFlg9z0Ta4qAQNjQbP1wsSt/oKkmZ/4/WWdMUpw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-arrow-functions@^7.0.0", "@babel/plugin-transform-arrow-functions@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.17.12.tgz#dddd783b473b1b1537ef46423e3944ff24898c45" + integrity sha512-PHln3CNi/49V+mza4xMwrg+WGYevSF1oaiXaC2EQfdp4HWlSjRsrDXWJiQBKpP7749u6vQ9mcry2uuFOv5CXvA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-async-to-generator@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.17.12.tgz#dbe5511e6b01eee1496c944e35cdfe3f58050832" + integrity sha512-J8dbrWIOO3orDzir57NRsjg4uxucvhby0L/KZuGsWDj0g7twWK3g7JhJhOrXtuXiw8MeiSdJ3E0OW9H8LYEzLQ== + dependencies: + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-remap-async-to-generator" "^7.16.8" + +"@babel/plugin-transform-block-scoped-functions@^7.0.0", "@babel/plugin-transform-block-scoped-functions@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.16.7.tgz#4d0d57d9632ef6062cdf354bb717102ee042a620" + integrity sha512-JUuzlzmF40Z9cXyytcbZEZKckgrQzChbQJw/5PuEHYeqzCsvebDx0K0jWnIIVcmmDOAVctCgnYs0pMcrYj2zJg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-block-scoping@^7.0.0", "@babel/plugin-transform-block-scoping@^7.17.12": + version "7.18.4" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.18.4.tgz#7988627b3e9186a13e4d7735dc9c34a056613fb9" + integrity sha512-+Hq10ye+jlvLEogSOtq4mKvtk7qwcUQ1f0Mrueai866C82f844Yom2cttfJdMdqRLTxWpsbfbkIkOIfovyUQXw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-classes@^7.0.0", "@babel/plugin-transform-classes@^7.15.4", "@babel/plugin-transform-classes@^7.17.12": + version "7.18.4" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-classes/-/plugin-transform-classes-7.18.4.tgz#51310b812a090b846c784e47087fa6457baef814" + integrity sha512-e42NSG2mlKWgxKUAD9EJJSkZxR67+wZqzNxLSpc51T8tRU5SLFHsPmgYR5yr7sdgX4u+iHA1C5VafJ6AyImV3A== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-optimise-call-expression" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-replace-supers" "^7.18.2" + "@babel/helper-split-export-declaration" "^7.16.7" + globals "^11.1.0" + +"@babel/plugin-transform-computed-properties@^7.0.0", "@babel/plugin-transform-computed-properties@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.17.12.tgz#bca616a83679698f3258e892ed422546e531387f" + integrity sha512-a7XINeplB5cQUWMg1E/GI1tFz3LfK021IjV1rj1ypE+R7jHm+pIHmHl25VNkZxtx9uuYp7ThGk8fur1HHG7PgQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-destructuring@^7.0.0", "@babel/plugin-transform-destructuring@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.18.0.tgz#dc4f92587e291b4daa78aa20cc2d7a63aa11e858" + integrity sha512-Mo69klS79z6KEfrLg/1WkmVnB8javh75HX4pi2btjvlIoasuxilEyjtsQW6XPrubNd7AQy0MMaNIaQE4e7+PQw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-dotall-regex@^7.16.7", "@babel/plugin-transform-dotall-regex@^7.4.4": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.16.7.tgz#6b2d67686fab15fb6a7fd4bd895d5982cfc81241" + integrity sha512-Lyttaao2SjZF6Pf4vk1dVKv8YypMpomAbygW+mU5cYP3S5cWTfCJjG8xV6CFdzGFlfWK81IjL9viiTvpb6G7gQ== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-duplicate-keys@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.17.12.tgz#a09aa709a3310013f8e48e0e23bc7ace0f21477c" + integrity sha512-EA5eYFUG6xeerdabina/xIoB95jJ17mAkR8ivx6ZSu9frKShBjpOGZPn511MTDTkiCO+zXnzNczvUM69YSf3Zw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-exponentiation-operator@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.16.7.tgz#efa9862ef97e9e9e5f653f6ddc7b665e8536fe9b" + integrity sha512-8UYLSlyLgRixQvlYH3J2ekXFHDFLQutdy7FfFAMm3CPZ6q9wHCwnUyiXpQCe3gVVnQlHc5nsuiEVziteRNTXEA== + dependencies: + "@babel/helper-builder-binary-assignment-operator-visitor" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-flow-strip-types@^7.0.0": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-flow-strip-types/-/plugin-transform-flow-strip-types-7.17.12.tgz#5e070f99a4152194bd9275de140e83a92966cab3" + integrity sha512-g8cSNt+cHCpG/uunPQELdq/TeV3eg1OLJYwxypwHtAWo9+nErH3lQx9CSO2uI9lF74A0mR0t4KoMjs1snSgnTw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-flow" "^7.17.12" + +"@babel/plugin-transform-for-of@^7.0.0", "@babel/plugin-transform-for-of@^7.18.1": + version "7.18.1" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.18.1.tgz#ed14b657e162b72afbbb2b4cdad277bf2bb32036" + integrity sha512-+TTB5XwvJ5hZbO8xvl2H4XaMDOAK57zF4miuC9qQJgysPNEAZZ9Z69rdF5LJkozGdZrjBIUAIyKUWRMmebI7vg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-function-name@^7.0.0", "@babel/plugin-transform-function-name@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.16.7.tgz#5ab34375c64d61d083d7d2f05c38d90b97ec65cf" + integrity sha512-SU/C68YVwTRxqWj5kgsbKINakGag0KTgq9f2iZEXdStoAbOzLHEBRYzImmA6yFo8YZhJVflvXmIHUO7GWHmxxA== + dependencies: + "@babel/helper-compilation-targets" "^7.16.7" + "@babel/helper-function-name" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-literals@^7.0.0", "@babel/plugin-transform-literals@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-literals/-/plugin-transform-literals-7.17.12.tgz#97131fbc6bbb261487105b4b3edbf9ebf9c830ae" + integrity sha512-8iRkvaTjJciWycPIZ9k9duu663FT7VrBdNqNgxnVXEFwOIp55JWcZd23VBRySYbnS3PwQ3rGiabJBBBGj5APmQ== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-member-expression-literals@^7.0.0", "@babel/plugin-transform-member-expression-literals@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.16.7.tgz#6e5dcf906ef8a098e630149d14c867dd28f92384" + integrity sha512-mBruRMbktKQwbxaJof32LT9KLy2f3gH+27a5XSuXo6h7R3vqltl0PgZ80C8ZMKw98Bf8bqt6BEVi3svOh2PzMw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-modules-amd@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.18.0.tgz#7ef1002e67e36da3155edc8bf1ac9398064c02ed" + integrity sha512-h8FjOlYmdZwl7Xm2Ug4iX2j7Qy63NANI+NQVWQzv6r25fqgg7k2dZl03p95kvqNclglHs4FZ+isv4p1uXMA+QA== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-commonjs@^7.0.0", "@babel/plugin-transform-modules-commonjs@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.18.2.tgz#1aa8efa2e2a6e818b6a7f2235fceaf09bdb31e9e" + integrity sha512-f5A865gFPAJAEE0K7F/+nm5CmAE3y8AWlMBG9unu5j9+tk50UQVK0QS8RNxSp7MJf0wh97uYyLWt3Zvu71zyOQ== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-simple-access" "^7.18.2" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-systemjs@^7.18.0": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.18.5.tgz#87f11c44fbfd3657be000d4897e192d9cb535996" + integrity sha512-SEewrhPpcqMF1V7DhnEbhVJLrC+nnYfe1E0piZMZXBpxi9WvZqWGwpsk7JYP7wPWeqaBh4gyKlBhHJu3uz5g4Q== + dependencies: + "@babel/helper-hoist-variables" "^7.16.7" + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-identifier" "^7.16.7" + babel-plugin-dynamic-import-node "^2.3.3" + +"@babel/plugin-transform-modules-umd@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.18.0.tgz#56aac64a2c2a1922341129a4597d1fd5c3ff020f" + integrity sha512-d/zZ8I3BWli1tmROLxXLc9A6YXvGK8egMxHp+E/rRwMh1Kip0AP77VwZae3snEJ33iiWwvNv2+UIIhfalqhzZA== + dependencies: + "@babel/helper-module-transforms" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-named-capturing-groups-regex@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.17.12.tgz#9c4a5a5966e0434d515f2675c227fd8cc8606931" + integrity sha512-vWoWFM5CKaTeHrdUJ/3SIOTRV+MBVGybOC9mhJkaprGNt5demMymDW24yC74avb915/mIRe3TgNb/d8idvnCRA== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.17.12" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-new-target@^7.17.12": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.18.5.tgz#8c228c4a07501dd12c95c5f23d1622131cc23931" + integrity sha512-TuRL5uGW4KXU6OsRj+mLp9BM7pO8e7SGNTEokQRRxHFkXYMFiy2jlKSZPFtI/mKORDzciH+hneskcSOp0gU8hg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-object-super@^7.0.0", "@babel/plugin-transform-object-super@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.16.7.tgz#ac359cf8d32cf4354d27a46867999490b6c32a94" + integrity sha512-14J1feiQVWaGvRxj2WjyMuXS2jsBkgB3MdSN5HuC2G5nRspa5RK9COcs82Pwy5BuGcjb+fYaUj94mYcOj7rCvw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/helper-replace-supers" "^7.16.7" + +"@babel/plugin-transform-parameters@^7.0.0", "@babel/plugin-transform-parameters@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.17.12.tgz#eb467cd9586ff5ff115a9880d6fdbd4a846b7766" + integrity sha512-6qW4rWo1cyCdq1FkYri7AHpauchbGLXpdwnYsfxFb+KtddHENfsY5JZb35xUwkK5opOLcJ3BNd2l7PhRYGlwIA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-property-literals@^7.0.0", "@babel/plugin-transform-property-literals@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.16.7.tgz#2dadac85155436f22c696c4827730e0fe1057a55" + integrity sha512-z4FGr9NMGdoIl1RqavCqGG+ZuYjfZ/hkCIeuH6Do7tXmSm0ls11nYVSJqFEUOSJbDab5wC6lRE/w6YjVcr6Hqw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-react-display-name@^7.0.0", "@babel/plugin-transform-react-display-name@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.16.7.tgz#7b6d40d232f4c0f550ea348593db3b21e2404340" + integrity sha512-qgIg8BcZgd0G/Cz916D5+9kqX0c7nPZyXaP8R2tLNN5tkyIZdG5fEwBrxwplzSnjC1jvQmyMNVwUCZPcbGY7Pg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-react-jsx-development@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.16.7.tgz#43a00724a3ed2557ed3f276a01a929e6686ac7b8" + integrity sha512-RMvQWvpla+xy6MlBpPlrKZCMRs2AGiHOGHY3xRwl0pEeim348dDyxeH4xBsMPbIMhujeq7ihE702eM2Ew0Wo+A== + dependencies: + "@babel/plugin-transform-react-jsx" "^7.16.7" + +"@babel/plugin-transform-react-jsx@^7.0.0", "@babel/plugin-transform-react-jsx@^7.16.7", "@babel/plugin-transform-react-jsx@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.17.12.tgz#2aa20022709cd6a3f40b45d60603d5f269586dba" + integrity sha512-Lcaw8bxd1DKht3thfD4A12dqo1X16he1Lm8rIv8sTwjAYNInRS1qHa9aJoqvzpscItXvftKDCfaEQzwoVyXpEQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-jsx" "^7.17.12" + "@babel/types" "^7.17.12" + +"@babel/plugin-transform-react-pure-annotations@^7.16.7": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.18.0.tgz#ef82c8e310913f3522462c9ac967d395092f1954" + integrity sha512-6+0IK6ouvqDn9bmEG7mEyF/pwlJXVj5lwydybpyyH3D0A7Hftk+NCTdYjnLNZksn261xaOV5ksmp20pQEmc2RQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-regenerator@^7.18.0": + version "7.18.0" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.18.0.tgz#44274d655eb3f1af3f3a574ba819d3f48caf99d5" + integrity sha512-C8YdRw9uzx25HSIzwA7EM7YP0FhCe5wNvJbZzjVNHHPGVcDJ3Aie+qGYYdS1oVQgn+B3eAIJbWFLrJ4Jipv7nw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + regenerator-transform "^0.15.0" + +"@babel/plugin-transform-reserved-words@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.17.12.tgz#7dbd349f3cdffba751e817cf40ca1386732f652f" + integrity sha512-1KYqwbJV3Co03NIi14uEHW8P50Md6KqFgt0FfpHdK6oyAHQVTosgPuPSiWud1HX0oYJ1hGRRlk0fP87jFpqXZA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-runtime@^7.15.0": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.18.5.tgz#f4d3188ba6a8815793993c71c2c225d0ee1d7743" + integrity sha512-Q17hHxXr2fplrE+5BSC1j1Fo5cOA8YeP8XW3/1paI8MzF/faZGh0MaH1KC4jLAvqLPamQWHB5/B7KqSLY1kuHA== + dependencies: + "@babel/helper-module-imports" "^7.16.7" + "@babel/helper-plugin-utils" "^7.17.12" + babel-plugin-polyfill-corejs2 "^0.3.0" + babel-plugin-polyfill-corejs3 "^0.5.0" + babel-plugin-polyfill-regenerator "^0.3.0" + semver "^6.3.0" + +"@babel/plugin-transform-shorthand-properties@^7.0.0", "@babel/plugin-transform-shorthand-properties@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.16.7.tgz#e8549ae4afcf8382f711794c0c7b6b934c5fbd2a" + integrity sha512-hah2+FEnoRoATdIb05IOXf+4GzXYTq75TVhIn1PewihbpyrNWUt2JbudKQOETWw6QpLe+AIUpJ5MVLYTQbeeUg== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-spread@^7.0.0", "@babel/plugin-transform-spread@^7.14.6", "@babel/plugin-transform-spread@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-spread/-/plugin-transform-spread-7.17.12.tgz#c112cad3064299f03ea32afed1d659223935d1f5" + integrity sha512-9pgmuQAtFi3lpNUstvG9nGfk9DkrdmWNp9KeKPFmuZCpEnxRzYlS8JgwPjYj+1AWDOSvoGN0H30p1cBOmT/Svg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-skip-transparent-expression-wrappers" "^7.16.0" + +"@babel/plugin-transform-sticky-regex@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.16.7.tgz#c84741d4f4a38072b9a1e2e3fd56d359552e8660" + integrity sha512-NJa0Bd/87QV5NZZzTuZG5BPJjLYadeSZ9fO6oOUoL4iQx+9EEuw/eEM92SrsT19Yc2jgB1u1hsjqDtH02c3Drw== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-template-literals@^7.0.0", "@babel/plugin-transform-template-literals@^7.18.2": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.18.2.tgz#31ed6915721864847c48b656281d0098ea1add28" + integrity sha512-/cmuBVw9sZBGZVOMkpAEaVLwm4JmK2GZ1dFKOGGpMzEHWFmyZZ59lUU0PdRr8YNYeQdNzTDwuxP2X2gzydTc9g== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-typeof-symbol@^7.17.12": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.17.12.tgz#0f12f57ac35e98b35b4ed34829948d42bd0e6889" + integrity sha512-Q8y+Jp7ZdtSPXCThB6zjQ74N3lj0f6TDh1Hnf5B+sYlzQ8i5Pjp8gW0My79iekSpT4WnI06blqP6DT0OmaXXmw== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + +"@babel/plugin-transform-typescript@^7.17.12": + version "7.18.4" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.18.4.tgz#587eaf6a39edb8c06215e550dc939faeadd750bf" + integrity sha512-l4vHuSLUajptpHNEOUDEGsnpl9pfRLsN1XUoDQDD/YBuXTM+v37SHGS+c6n4jdcZy96QtuUuSvZYMLSSsjH8Mw== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.18.0" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/plugin-syntax-typescript" "^7.17.12" + +"@babel/plugin-transform-unicode-escapes@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.16.7.tgz#da8717de7b3287a2c6d659750c964f302b31ece3" + integrity sha512-TAV5IGahIz3yZ9/Hfv35TV2xEm+kaBDaZQCn2S/hG9/CZ0DktxJv9eKfPc7yYCvOYR4JGx1h8C+jcSOvgaaI/Q== + dependencies: + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/plugin-transform-unicode-regex@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.16.7.tgz#0f7aa4a501198976e25e82702574c34cfebe9ef2" + integrity sha512-oC5tYYKw56HO75KZVLQ+R/Nl3Hro9kf8iG0hXoaHP7tjAyCpvqBiSNe6vGrZni1Z6MggmUOC6A7VP7AVmw225Q== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.16.7" + "@babel/helper-plugin-utils" "^7.16.7" + +"@babel/preset-env@^7.15.4": + version "7.18.2" + resolved "https://registry.npmmirror.com/@babel/preset-env/-/preset-env-7.18.2.tgz#f47d3000a098617926e674c945d95a28cb90977a" + integrity sha512-PfpdxotV6afmXMU47S08F9ZKIm2bJIQ0YbAAtDfIENX7G1NUAXigLREh69CWDjtgUy7dYn7bsMzkgdtAlmS68Q== + dependencies: + "@babel/compat-data" "^7.17.10" + "@babel/helper-compilation-targets" "^7.18.2" + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression" "^7.17.12" + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining" "^7.17.12" + "@babel/plugin-proposal-async-generator-functions" "^7.17.12" + "@babel/plugin-proposal-class-properties" "^7.17.12" + "@babel/plugin-proposal-class-static-block" "^7.18.0" + "@babel/plugin-proposal-dynamic-import" "^7.16.7" + "@babel/plugin-proposal-export-namespace-from" "^7.17.12" + "@babel/plugin-proposal-json-strings" "^7.17.12" + "@babel/plugin-proposal-logical-assignment-operators" "^7.17.12" + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.17.12" + "@babel/plugin-proposal-numeric-separator" "^7.16.7" + "@babel/plugin-proposal-object-rest-spread" "^7.18.0" + "@babel/plugin-proposal-optional-catch-binding" "^7.16.7" + "@babel/plugin-proposal-optional-chaining" "^7.17.12" + "@babel/plugin-proposal-private-methods" "^7.17.12" + "@babel/plugin-proposal-private-property-in-object" "^7.17.12" + "@babel/plugin-proposal-unicode-property-regex" "^7.17.12" + "@babel/plugin-syntax-async-generators" "^7.8.4" + "@babel/plugin-syntax-class-properties" "^7.12.13" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + "@babel/plugin-syntax-import-assertions" "^7.17.12" + "@babel/plugin-syntax-json-strings" "^7.8.3" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + "@babel/plugin-syntax-top-level-await" "^7.14.5" + "@babel/plugin-transform-arrow-functions" "^7.17.12" + "@babel/plugin-transform-async-to-generator" "^7.17.12" + "@babel/plugin-transform-block-scoped-functions" "^7.16.7" + "@babel/plugin-transform-block-scoping" "^7.17.12" + "@babel/plugin-transform-classes" "^7.17.12" + "@babel/plugin-transform-computed-properties" "^7.17.12" + "@babel/plugin-transform-destructuring" "^7.18.0" + "@babel/plugin-transform-dotall-regex" "^7.16.7" + "@babel/plugin-transform-duplicate-keys" "^7.17.12" + "@babel/plugin-transform-exponentiation-operator" "^7.16.7" + "@babel/plugin-transform-for-of" "^7.18.1" + "@babel/plugin-transform-function-name" "^7.16.7" + "@babel/plugin-transform-literals" "^7.17.12" + "@babel/plugin-transform-member-expression-literals" "^7.16.7" + "@babel/plugin-transform-modules-amd" "^7.18.0" + "@babel/plugin-transform-modules-commonjs" "^7.18.2" + "@babel/plugin-transform-modules-systemjs" "^7.18.0" + "@babel/plugin-transform-modules-umd" "^7.18.0" + "@babel/plugin-transform-named-capturing-groups-regex" "^7.17.12" + "@babel/plugin-transform-new-target" "^7.17.12" + "@babel/plugin-transform-object-super" "^7.16.7" + "@babel/plugin-transform-parameters" "^7.17.12" + "@babel/plugin-transform-property-literals" "^7.16.7" + "@babel/plugin-transform-regenerator" "^7.18.0" + "@babel/plugin-transform-reserved-words" "^7.17.12" + "@babel/plugin-transform-shorthand-properties" "^7.16.7" + "@babel/plugin-transform-spread" "^7.17.12" + "@babel/plugin-transform-sticky-regex" "^7.16.7" + "@babel/plugin-transform-template-literals" "^7.18.2" + "@babel/plugin-transform-typeof-symbol" "^7.17.12" + "@babel/plugin-transform-unicode-escapes" "^7.16.7" + "@babel/plugin-transform-unicode-regex" "^7.16.7" + "@babel/preset-modules" "^0.1.5" + "@babel/types" "^7.18.2" + babel-plugin-polyfill-corejs2 "^0.3.0" + babel-plugin-polyfill-corejs3 "^0.5.0" + babel-plugin-polyfill-regenerator "^0.3.0" + core-js-compat "^3.22.1" + semver "^6.3.0" + +"@babel/preset-modules@^0.1.5": + version "0.1.5" + resolved "https://registry.npmmirror.com/@babel/preset-modules/-/preset-modules-0.1.5.tgz#ef939d6e7f268827e1841638dc6ff95515e115d9" + integrity sha512-A57th6YRG7oR3cq/yt/Y84MvGgE0eJG2F1JLhKuyG+jFxEgrd/HAMJatiFtmOiZurz+0DkrvbheCLaV5f2JfjA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@babel/plugin-proposal-unicode-property-regex" "^7.4.4" + "@babel/plugin-transform-dotall-regex" "^7.4.4" + "@babel/types" "^7.4.4" + esutils "^2.0.2" + +"@babel/preset-react@^7.14.0": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/preset-react/-/preset-react-7.17.12.tgz#62adbd2d1870c0de3893095757ed5b00b492ab3d" + integrity sha512-h5U+rwreXtZaRBEQhW1hOJLMq8XNJBQ/9oymXiCXTuT/0uOwpbT0gUt+sXeOqoXBgNuUKI7TaObVwoEyWkpFgA== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-transform-react-display-name" "^7.16.7" + "@babel/plugin-transform-react-jsx" "^7.17.12" + "@babel/plugin-transform-react-jsx-development" "^7.16.7" + "@babel/plugin-transform-react-pure-annotations" "^7.16.7" + +"@babel/preset-typescript@^7.15.0", "@babel/preset-typescript@^7.16.7": + version "7.17.12" + resolved "https://registry.npmmirror.com/@babel/preset-typescript/-/preset-typescript-7.17.12.tgz#40269e0a0084d56fc5731b6c40febe1c9a4a3e8c" + integrity sha512-S1ViF8W2QwAKUGJXxP9NAfNaqGDdEBJKpYkxHf5Yy2C4NPPzXGeR3Lhk7G8xJaaLcFTRfNjVbtbVtm8Gb0mqvg== + dependencies: + "@babel/helper-plugin-utils" "^7.17.12" + "@babel/helper-validator-option" "^7.16.7" + "@babel/plugin-transform-typescript" "^7.17.12" + +"@babel/runtime-corejs3@^7.10.2": + version "7.18.3" + resolved "https://registry.npmmirror.com/@babel/runtime-corejs3/-/runtime-corejs3-7.18.3.tgz#52f0241a31e0ec61a6187530af6227c2846bd60c" + integrity sha512-l4ddFwrc9rnR+EJsHsh+TJ4A35YqQz/UqcjtlX2ov53hlJYG5CxtQmNZxyajwDVmCxwy++rtvGU5HazCK4W41Q== + dependencies: + core-js-pure "^3.20.2" + regenerator-runtime "^0.13.4" + +"@babel/runtime@^7.0.0", "@babel/runtime@^7.10.0", "@babel/runtime@^7.10.1", "@babel/runtime@^7.10.2", "@babel/runtime@^7.10.4", "@babel/runtime@^7.11.1", "@babel/runtime@^7.11.2", "@babel/runtime@^7.12.5", "@babel/runtime@^7.13.10", "@babel/runtime@^7.15.4", "@babel/runtime@^7.16.3", "@babel/runtime@^7.18.0", "@babel/runtime@^7.18.3", "@babel/runtime@^7.3.4", "@babel/runtime@^7.7.2", "@babel/runtime@^7.8.4", "@babel/runtime@^7.9.2": + version "7.18.3" + resolved "https://registry.npmmirror.com/@babel/runtime/-/runtime-7.18.3.tgz#c7b654b57f6f63cf7f8b418ac9ca04408c4579f4" + integrity sha512-38Y8f7YUhce/K7RMwTp7m0uCumpv9hZkitCbBClqQIow1qSbCvGkcegKOXpEWCQLfWmevgRiWokZ1GkpfhbZug== + dependencies: + regenerator-runtime "^0.13.4" + +"@babel/template@^7.16.7": + version "7.16.7" + resolved "https://registry.npmmirror.com/@babel/template/-/template-7.16.7.tgz#8d126c8701fde4d66b264b3eba3d96f07666d155" + integrity sha512-I8j/x8kHUrbYRTUxXrrMbfCa7jxkE7tZre39x3kjr9hvI82cK1FfqLygotcWN5kdPGWcLdWMHpSBavse5tWw3w== + dependencies: + "@babel/code-frame" "^7.16.7" + "@babel/parser" "^7.16.7" + "@babel/types" "^7.16.7" + +"@babel/traverse@^7.13.0", "@babel/traverse@^7.14.0", "@babel/traverse@^7.15.4", "@babel/traverse@^7.16.8", "@babel/traverse@^7.18.0", "@babel/traverse@^7.18.2", "@babel/traverse@^7.18.5": + version "7.18.5" + resolved "https://registry.npmmirror.com/@babel/traverse/-/traverse-7.18.5.tgz#94a8195ad9642801837988ab77f36e992d9a20cd" + integrity sha512-aKXj1KT66sBj0vVzk6rEeAO6Z9aiiQ68wfDgge3nHhA/my6xMM/7HGQUNumKZaoa2qUPQ5whJG9aAifsxUKfLA== + dependencies: + "@babel/code-frame" "^7.16.7" + "@babel/generator" "^7.18.2" + "@babel/helper-environment-visitor" "^7.18.2" + "@babel/helper-function-name" "^7.17.9" + "@babel/helper-hoist-variables" "^7.16.7" + "@babel/helper-split-export-declaration" "^7.16.7" + "@babel/parser" "^7.18.5" + "@babel/types" "^7.18.4" + debug "^4.1.0" + globals "^11.1.0" + +"@babel/types@^7.0.0", "@babel/types@^7.0.0-beta.49", "@babel/types@^7.15.4", "@babel/types@^7.16.0", "@babel/types@^7.16.7", "@babel/types@^7.16.8", "@babel/types@^7.17.0", "@babel/types@^7.17.12", "@babel/types@^7.18.0", "@babel/types@^7.18.2", "@babel/types@^7.18.4", "@babel/types@^7.4.4": + version "7.18.4" + resolved "https://registry.npmmirror.com/@babel/types/-/types-7.18.4.tgz#27eae9b9fd18e9dccc3f9d6ad051336f307be354" + integrity sha512-ThN1mBcMq5pG/Vm2IcBmPPfyPXbd8S02rS+OBIDENdufvqC7Z/jHPCv9IcP01277aKtDI8g/2XysBN4hA8niiw== + dependencies: + "@babel/helper-validator-identifier" "^7.16.7" + to-fast-properties "^2.0.0" + +"@builder.io/partytown@^0.5.2": + version "0.5.4" + resolved "https://registry.npmmirror.com/@builder.io/partytown/-/partytown-0.5.4.tgz#1a89069978734e132fa4a59414ddb64e4b94fde7" + integrity sha512-qnikpQgi30AS01aFlNQV6l8/qdZIcP76mp90ti+u4rucXHsn4afSKivQXApqxvrQG9+Ibv45STyvHizvxef/7A== + +"@ctrl/tinycolor@^3.4.0": + version "3.4.1" + resolved "https://registry.npmmirror.com/@ctrl/tinycolor/-/tinycolor-3.4.1.tgz#75b4c27948c81e88ccd3a8902047bcd797f38d32" + integrity sha512-ej5oVy6lykXsvieQtqZxCOaLT+xD4+QNarq78cIYISHmZXshCvROLudpQN3lfL8G0NL7plMSSK+zlyvCaIJ4Iw== + +"@emotion/cache@11.7.1": + version "11.7.1" + resolved "https://registry.npmmirror.com/@emotion/cache/-/cache-11.7.1.tgz#08d080e396a42e0037848214e8aa7bf879065539" + integrity sha512-r65Zy4Iljb8oyjtLeCuBH8Qjiy107dOYC6SJq7g7GV5UCQWMObY4SJDPGFjiiVpPrOJ2hmJOoBiYTC7hwx9E2A== + dependencies: + "@emotion/memoize" "^0.7.4" + "@emotion/sheet" "^1.1.0" + "@emotion/utils" "^1.0.0" + "@emotion/weak-memoize" "^0.2.5" + stylis "4.0.13" + +"@emotion/cache@^11.7.1": + version "11.9.3" + resolved "https://registry.npmmirror.com/@emotion/cache/-/cache-11.9.3.tgz#96638449f6929fd18062cfe04d79b29b44c0d6cb" + integrity sha512-0dgkI/JKlCXa+lEXviaMtGBL0ynpx4osh7rjOXE71q9bIF8G+XhJgvi+wDu0B0IdCVx37BffiwXlN9I3UuzFvg== + dependencies: + "@emotion/memoize" "^0.7.4" + "@emotion/sheet" "^1.1.1" + "@emotion/utils" "^1.0.0" + "@emotion/weak-memoize" "^0.2.5" + stylis "4.0.13" + +"@emotion/hash@^0.8.0": + version "0.8.0" + resolved "https://registry.npmmirror.com/@emotion/hash/-/hash-0.8.0.tgz#bbbff68978fefdbe68ccb533bc8cbe1d1afb5413" + integrity sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow== + +"@emotion/memoize@^0.7.4": + version "0.7.5" + resolved "https://registry.npmmirror.com/@emotion/memoize/-/memoize-0.7.5.tgz#2c40f81449a4e554e9fc6396910ed4843ec2be50" + integrity sha512-igX9a37DR2ZPGYtV6suZ6whr8pTFtyHL3K/oLUotxpSVO2ASaprmAe2Dkq7tBo7CRY7MMDrAa9nuQP9/YG8FxQ== + +"@emotion/react@11.7.1": + version "11.7.1" + resolved "https://registry.npmmirror.com/@emotion/react/-/react-11.7.1.tgz#3f800ce9b20317c13e77b8489ac4a0b922b2fe07" + integrity sha512-DV2Xe3yhkF1yT4uAUoJcYL1AmrnO5SVsdfvu+fBuS7IbByDeTVx9+wFmvx9Idzv7/78+9Mgx2Hcmr7Fex3tIyw== + dependencies: + "@babel/runtime" "^7.13.10" + "@emotion/cache" "^11.7.1" + "@emotion/serialize" "^1.0.2" + "@emotion/sheet" "^1.1.0" + "@emotion/utils" "^1.0.0" + "@emotion/weak-memoize" "^0.2.5" + hoist-non-react-statics "^3.3.1" + +"@emotion/serialize@1.0.2": + version "1.0.2" + resolved "https://registry.npmmirror.com/@emotion/serialize/-/serialize-1.0.2.tgz#77cb21a0571c9f68eb66087754a65fa97bfcd965" + integrity sha512-95MgNJ9+/ajxU7QIAruiOAdYNjxZX7G2mhgrtDWswA21VviYIRP1R5QilZ/bDY42xiKsaktP4egJb3QdYQZi1A== + dependencies: + "@emotion/hash" "^0.8.0" + "@emotion/memoize" "^0.7.4" + "@emotion/unitless" "^0.7.5" + "@emotion/utils" "^1.0.0" + csstype "^3.0.2" + +"@emotion/serialize@^1.0.2": + version "1.0.4" + resolved "https://registry.npmmirror.com/@emotion/serialize/-/serialize-1.0.4.tgz#ff31fd11bb07999611199c2229e152faadc21a3c" + integrity sha512-1JHamSpH8PIfFwAMryO2bNka+y8+KA5yga5Ocf2d7ZEiJjb7xlLW7aknBGZqJLajuLOvJ+72vN+IBSwPlXD1Pg== + dependencies: + "@emotion/hash" "^0.8.0" + "@emotion/memoize" "^0.7.4" + "@emotion/unitless" "^0.7.5" + "@emotion/utils" "^1.0.0" + csstype "^3.0.2" + +"@emotion/server@11.4.0": + version "11.4.0" + resolved "https://registry.npmmirror.com/@emotion/server/-/server-11.4.0.tgz#3ae1d74cb31c7d013c3c76e88c0c4439076e9f66" + integrity sha512-IHovdWA3V0DokzxLtUNDx4+hQI82zUXqQFcVz/om2t44O0YSc+NHB+qifnyAOoQwt3SXcBTgaSntobwUI9gnfA== + dependencies: + "@emotion/utils" "^1.0.0" + html-tokenize "^2.0.0" + multipipe "^1.0.2" + through "^2.3.8" + +"@emotion/sheet@^1.1.0", "@emotion/sheet@^1.1.1": + version "1.1.1" + resolved "https://registry.npmmirror.com/@emotion/sheet/-/sheet-1.1.1.tgz#015756e2a9a3c7c5f11d8ec22966a8dbfbfac787" + integrity sha512-J3YPccVRMiTZxYAY0IOq3kd+hUP8idY8Kz6B/Cyo+JuXq52Ek+zbPbSQUrVQp95aJ+lsAW7DPL1P2Z+U1jGkKA== + +"@emotion/unitless@^0.7.5": + version "0.7.5" + resolved "https://registry.npmmirror.com/@emotion/unitless/-/unitless-0.7.5.tgz#77211291c1900a700b8a78cfafda3160d76949ed" + integrity sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg== + +"@emotion/utils@1.0.0": + version "1.0.0" + resolved "https://registry.npmmirror.com/@emotion/utils/-/utils-1.0.0.tgz#abe06a83160b10570816c913990245813a2fd6af" + integrity sha512-mQC2b3XLDs6QCW+pDQDiyO/EdGZYOygE8s5N5rrzjSI4M3IejPE/JPndCBwRT9z982aqQNi6beWs1UeayrQxxA== + +"@emotion/utils@^1.0.0": + version "1.1.0" + resolved "https://registry.npmmirror.com/@emotion/utils/-/utils-1.1.0.tgz#86b0b297f3f1a0f2bdb08eeac9a2f49afd40d0cf" + integrity sha512-iRLa/Y4Rs5H/f2nimczYmS5kFJEbpiVvgN3XVfZ022IYhuNA1IRSHEizcof88LtCTXtl9S2Cxt32KgaXEu72JQ== + +"@emotion/weak-memoize@^0.2.5": + version "0.2.5" + resolved "https://registry.npmmirror.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz#8eed982e2ee6f7f4e44c253e12962980791efd46" + integrity sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA== + +"@endemolshinegroup/cosmiconfig-typescript-loader@3.0.2": + version "3.0.2" + resolved "https://registry.npmmirror.com/@endemolshinegroup/cosmiconfig-typescript-loader/-/cosmiconfig-typescript-loader-3.0.2.tgz#eea4635828dde372838b0909693ebd9aafeec22d" + integrity sha512-QRVtqJuS1mcT56oHpVegkKBlgtWjXw/gHNWO3eL9oyB5Sc7HBoc2OLG/nYpVfT/Jejvo3NUrD0Udk7XgoyDKkA== + dependencies: + lodash.get "^4" + make-error "^1" + ts-node "^9" + tslib "^2" + +"@eslint/eslintrc@^0.4.3": + version "0.4.3" + resolved "https://registry.npmmirror.com/@eslint/eslintrc/-/eslintrc-0.4.3.tgz#9e42981ef035beb3dd49add17acb96e8ff6f394c" + integrity sha512-J6KFFz5QCYUJq3pf0mjEcCJVERbzv71PUIDczuh9JkwGEzced6CO5ADLHB1rbf/+oPBtoPfMYNOpGDzCANlbXw== + dependencies: + ajv "^6.12.4" + debug "^4.1.1" + espree "^7.3.0" + globals "^13.9.0" + ignore "^4.0.6" + import-fresh "^3.2.1" + js-yaml "^3.13.1" + minimatch "^3.0.4" + strip-json-comments "^3.1.1" + +"@gatsbyjs/parcel-namer-relative-to-cwd@^1.2.0": + version "1.2.0" + resolved "https://registry.npmmirror.com/@gatsbyjs/parcel-namer-relative-to-cwd/-/parcel-namer-relative-to-cwd-1.2.0.tgz#3ebc0e5bbe90e7a897d5427c2b67ffe26730228f" + integrity sha512-4iIdquFDl+b+U8Ng0dg6dCtxB/cnH27ERrlQQlxfdaWe8e9CLo8aWc6u3UeuHwNJixBFOUbOgEFaA5qCUPwLCQ== + dependencies: + "@babel/runtime" "^7.18.0" + "@parcel/plugin" "2.6.0" + gatsby-core-utils "^3.17.0" + +"@gatsbyjs/potrace@^2.2.0": + version "2.2.0" + resolved "https://registry.npmmirror.com/@gatsbyjs/potrace/-/potrace-2.2.0.tgz#56e88e420c65bb3065caa27772c6acd1c16c4ec4" + integrity sha512-/RiLVFJA+CIYhceb6XL1kD1GZ5E2JBX38pld0fiGNiNwLl+Bb7TYZR72aQvcs3v+NOrSjbagUiCnIHYmEW4F7w== + dependencies: + jimp "^0.16.1" + +"@gatsbyjs/reach-router@^1.3.6": + version "1.3.7" + resolved "https://registry.npmmirror.com/@gatsbyjs/reach-router/-/reach-router-1.3.7.tgz#d32029f2b4d91bb6977e7fd605237e3a5db20096" + integrity sha512-KQ5FvMb4BZUlSo+yQgd4t4WB8vkVPWfKjTpSl+Bx/FZhU6OL4lpwgfX7fXAY/18DogqyJCFiNAjV5eo3rQ5Alw== + dependencies: + invariant "^2.2.3" + prop-types "^15.6.1" + react-lifecycles-compat "^3.0.4" + +"@gatsbyjs/webpack-hot-middleware@^2.25.2": + version "2.25.3" + resolved "https://registry.npmmirror.com/@gatsbyjs/webpack-hot-middleware/-/webpack-hot-middleware-2.25.3.tgz#a00c5f526cbe178330b967f7ed6a487751ffd795" + integrity sha512-ul17OZ8Dlw+ATRbnuU+kwxuAlq9lKbYz/2uBS1FLCdgoPTF1H2heP7HbUbgfMZbfRQNcCG2rMscMnr32ritCDw== + dependencies: + ansi-html-community "0.0.8" + html-entities "^2.3.3" + strip-ansi "^6.0.0" + +"@graphql-codegen/add@^3.1.1": + version "3.1.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/add/-/add-3.1.1.tgz#e161ff1c7cdf74ce20b32f75f640f9592b9a18ca" + integrity sha512-XkVwcqosa0CVBlL1HaQT0gp+EUfhuQE3LzrEpzMQLwchxaj/NPVYtOJL6MUHaYDsHzLqxWrufjfbeB3y2NQgRw== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.3.2" + tslib "~2.3.0" + +"@graphql-codegen/core@^2.5.1": + version "2.5.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/core/-/core-2.5.1.tgz#e3d50d3449b8c58b74ea08e97faf656a1b7fc8a1" + integrity sha512-alctBVl2hMnBXDLwkgmnFPrZVIiBDsWJSmxJcM4GKg1PB23+xuov35GE47YAyAhQItE1B1fbYnbb1PtGiDZ4LA== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.4.1" + "@graphql-tools/schema" "^8.1.2" + "@graphql-tools/utils" "^8.1.1" + tslib "~2.3.0" + +"@graphql-codegen/plugin-helpers@^2.3.2", "@graphql-codegen/plugin-helpers@^2.4.0", "@graphql-codegen/plugin-helpers@^2.4.1", "@graphql-codegen/plugin-helpers@^2.4.2": + version "2.4.2" + resolved "https://registry.npmmirror.com/@graphql-codegen/plugin-helpers/-/plugin-helpers-2.4.2.tgz#e4f6b74dddcf8a9974fef5ce48562ae0980f9fed" + integrity sha512-LJNvwAPv/sKtI3RnRDm+nPD+JeOfOuSOS4FFIpQCMUCyMnFcchV/CPTTv7tT12fLUpEg6XjuFfDBvOwndti30Q== + dependencies: + "@graphql-tools/utils" "^8.5.2" + change-case-all "1.0.14" + common-tags "1.8.2" + import-from "4.0.0" + lodash "~4.17.0" + tslib "~2.3.0" + +"@graphql-codegen/schema-ast@^2.4.1": + version "2.4.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/schema-ast/-/schema-ast-2.4.1.tgz#ad742b53e32f7a2fbff8ea8a91ba7e617e6ef236" + integrity sha512-bIWlKk/ShoVJfghA4Rt1OWnd34/dQmZM/vAe6fu6QKyOh44aAdqPtYQ2dbTyFXoknmu504etKJGEDllYNUJRfg== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.3.2" + "@graphql-tools/utils" "^8.1.1" + tslib "~2.3.0" + +"@graphql-codegen/typescript-operations@^2.3.5": + version "2.4.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/typescript-operations/-/typescript-operations-2.4.1.tgz#bef0c4d3acf70879e2be24d73515df523a2bcdc2" + integrity sha512-NIHn4T5wxQ45PtKJ19q+AOqR4f2jW8UlMfQD34RG7vTuD42g+Y00GupYFTyzE6bsUAIhsVgJ4QJmP+taipBKAQ== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.4.0" + "@graphql-codegen/typescript" "^2.5.0" + "@graphql-codegen/visitor-plugin-common" "2.9.0" + auto-bind "~4.0.0" + tslib "~2.4.0" + +"@graphql-codegen/typescript@^2.4.8", "@graphql-codegen/typescript@^2.5.0": + version "2.5.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/typescript/-/typescript-2.5.1.tgz#5131485ff3ac83d5bc0aae11a2af0c9bdc08854c" + integrity sha512-D/9V2VfVIE4Mu5UiMGQtxyFU5xe1ZkAZi8g/IsqymW8rqlhTwsGhtk4JR55qPfOYxR8G94RJSJpzgNakRneytw== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.4.0" + "@graphql-codegen/schema-ast" "^2.4.1" + "@graphql-codegen/visitor-plugin-common" "2.9.1" + auto-bind "~4.0.0" + tslib "~2.4.0" + +"@graphql-codegen/visitor-plugin-common@2.9.0": + version "2.9.0" + resolved "https://registry.npmmirror.com/@graphql-codegen/visitor-plugin-common/-/visitor-plugin-common-2.9.0.tgz#a13556c805e72b63ee9d80fb437b9ba2eb0a9fa2" + integrity sha512-w353lPpXlgsQT6dxfcHw3hQdodGnfSwsxofHVfROqrIBZIjH0kNiFHOt8rnewYkLGEa75bWDVaM1ofyXEaN+3w== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.4.0" + "@graphql-tools/optimize" "^1.0.1" + "@graphql-tools/relay-operation-optimizer" "^6.3.7" + "@graphql-tools/utils" "^8.3.0" + auto-bind "~4.0.0" + change-case-all "1.0.14" + dependency-graph "^0.11.0" + graphql-tag "^2.11.0" + parse-filepath "^1.0.2" + tslib "~2.4.0" + +"@graphql-codegen/visitor-plugin-common@2.9.1": + version "2.9.1" + resolved "https://registry.npmmirror.com/@graphql-codegen/visitor-plugin-common/-/visitor-plugin-common-2.9.1.tgz#17dfe33e19e846e7475ab9d4ff43de5130e18397" + integrity sha512-j9eGOSGt+sJcwv0ijhZiQ2cF/0ponscekNVoF+vHdOT4RB0qgOQxykPBk6EbKxIHECnkdV8ARdPVTA21A93/QQ== + dependencies: + "@graphql-codegen/plugin-helpers" "^2.4.0" + "@graphql-tools/optimize" "^1.0.1" + "@graphql-tools/relay-operation-optimizer" "^6.4.14" + "@graphql-tools/utils" "^8.3.0" + auto-bind "~4.0.0" + change-case-all "1.0.14" + dependency-graph "^0.11.0" + graphql-tag "^2.11.0" + parse-filepath "^1.0.2" + tslib "~2.4.0" + +"@graphql-tools/batch-execute@^7.1.2": + version "7.1.2" + resolved "https://registry.npmmirror.com/@graphql-tools/batch-execute/-/batch-execute-7.1.2.tgz#35ba09a1e0f80f34f1ce111d23c40f039d4403a0" + integrity sha512-IuR2SB2MnC2ztA/XeTMTfWcA0Wy7ZH5u+nDkDNLAdX+AaSyDnsQS35sCmHqG0VOGTl7rzoyBWLCKGwSJplgtwg== + dependencies: + "@graphql-tools/utils" "^7.7.0" + dataloader "2.0.0" + tslib "~2.2.0" + value-or-promise "1.0.6" + +"@graphql-tools/code-file-loader@^7.2.14": + version "7.2.18" + resolved "https://registry.npmmirror.com/@graphql-tools/code-file-loader/-/code-file-loader-7.2.18.tgz#c6633aa5065c50bb667599b4ad7ae1a6c5c72531" + integrity sha512-zHJ2SPuWqK2/rlyxsb4maQo2locqNsZX3Dp5QoiXhUEsrf5vHkEHlp68ldcoFSzveBhXvIAdNrQVgVhzBQ3q7Q== + dependencies: + "@graphql-tools/graphql-tag-pluck" "7.2.10" + "@graphql-tools/utils" "8.6.13" + globby "^11.0.3" + tslib "^2.4.0" + unixify "^1.0.0" + +"@graphql-tools/delegate@^7.0.1", "@graphql-tools/delegate@^7.1.5": + version "7.1.5" + resolved "https://registry.npmmirror.com/@graphql-tools/delegate/-/delegate-7.1.5.tgz#0b027819b7047eff29bacbd5032e34a3d64bd093" + integrity sha512-bQu+hDd37e+FZ0CQGEEczmRSfQRnnXeUxI/0miDV+NV/zCbEdIJj5tYFNrKT03W6wgdqx8U06d8L23LxvGri/g== + dependencies: + "@ardatan/aggregate-error" "0.0.6" + "@graphql-tools/batch-execute" "^7.1.2" + "@graphql-tools/schema" "^7.1.5" + "@graphql-tools/utils" "^7.7.1" + dataloader "2.0.0" + tslib "~2.2.0" + value-or-promise "1.0.6" + +"@graphql-tools/graphql-file-loader@^6.0.0": + version "6.2.7" + resolved "https://registry.npmmirror.com/@graphql-tools/graphql-file-loader/-/graphql-file-loader-6.2.7.tgz#d3720f2c4f4bb90eb2a03a7869a780c61945e143" + integrity sha512-5k2SNz0W87tDcymhEMZMkd6/vs6QawDyjQXWtqkuLTBF3vxjxPD1I4dwHoxgWPIjjANhXybvulD7E+St/7s9TQ== + dependencies: + "@graphql-tools/import" "^6.2.6" + "@graphql-tools/utils" "^7.0.0" + tslib "~2.1.0" + +"@graphql-tools/graphql-tag-pluck@7.2.10": + version "7.2.10" + resolved "https://registry.npmmirror.com/@graphql-tools/graphql-tag-pluck/-/graphql-tag-pluck-7.2.10.tgz#f9dfeac8a8369de6a70157388e6cc6d1d984825d" + integrity sha512-j2f0Wzqy69XerNlTTTpSF1weLZN2z8NRrqP0lW/J3bKK9IgOy5eNDzcUUGujcn+MvjkpmjRaD4VFuxN75S2ozQ== + dependencies: + "@babel/parser" "^7.16.8" + "@babel/traverse" "^7.16.8" + "@babel/types" "^7.16.8" + "@graphql-tools/utils" "8.6.13" + tslib "^2.4.0" + +"@graphql-tools/import@^6.2.6": + version "6.6.17" + resolved "https://registry.npmmirror.com/@graphql-tools/import/-/import-6.6.17.tgz#fb5b5a75b8bd81e9abfd22b04894babf7c0f6ff3" + integrity sha512-rnKT2ZaFM+IbSFE0iOGG5sqdaDDv/XHHH43VIpV4ozryKoK9re3qrhEgfDOHaW47zMLGKrHLPCC/QGf0IpJquw== + dependencies: + "@graphql-tools/utils" "8.6.13" + resolve-from "5.0.0" + tslib "^2.4.0" + +"@graphql-tools/json-file-loader@^6.0.0": + version "6.2.6" + resolved "https://registry.npmmirror.com/@graphql-tools/json-file-loader/-/json-file-loader-6.2.6.tgz#830482cfd3721a0799cbf2fe5b09959d9332739a" + integrity sha512-CnfwBSY5926zyb6fkDBHnlTblHnHI4hoBALFYXnrg0Ev4yWU8B04DZl/pBRUc459VNgO2x8/mxGIZj2hPJG1EA== + dependencies: + "@graphql-tools/utils" "^7.0.0" + tslib "~2.0.1" + +"@graphql-tools/load@^6.0.0": + version "6.2.8" + resolved "https://registry.npmmirror.com/@graphql-tools/load/-/load-6.2.8.tgz#16900fb6e75e1d075cad8f7ea439b334feb0b96a" + integrity sha512-JpbyXOXd8fJXdBh2ta0Q4w8ia6uK5FHzrTNmcvYBvflFuWly2LDTk2abbSl81zKkzswQMEd2UIYghXELRg8eTA== + dependencies: + "@graphql-tools/merge" "^6.2.12" + "@graphql-tools/utils" "^7.5.0" + globby "11.0.3" + import-from "3.0.0" + is-glob "4.0.1" + p-limit "3.1.0" + tslib "~2.2.0" + unixify "1.0.0" + valid-url "1.0.9" + +"@graphql-tools/load@^7.5.10": + version "7.5.14" + resolved "https://registry.npmmirror.com/@graphql-tools/load/-/load-7.5.14.tgz#820bddc5b6f6172827cf43e1ee10e4317677cdc5" + integrity sha512-K7H4tKKGFliRyjbG92KCuv2fS2pHlRxkcNcDtuEQlA8dhthS9qGB14Ld4eHDuRq1RvHTS6mye5NE1alyY44K9g== + dependencies: + "@graphql-tools/schema" "8.3.14" + "@graphql-tools/utils" "8.6.13" + p-limit "3.1.0" + tslib "^2.4.0" + +"@graphql-tools/merge@6.0.0 - 6.2.14": + version "6.2.14" + resolved "https://registry.npmmirror.com/@graphql-tools/merge/-/merge-6.2.14.tgz#694e2a2785ba47558e5665687feddd2935e9d94e" + integrity sha512-RWT4Td0ROJai2eR66NHejgf8UwnXJqZxXgDWDI+7hua5vNA2OW8Mf9K1Wav1ZkjWnuRp4ztNtkZGie5ISw55ow== + dependencies: + "@graphql-tools/schema" "^7.0.0" + "@graphql-tools/utils" "^7.7.0" + tslib "~2.2.0" + +"@graphql-tools/merge@8.2.14": + version "8.2.14" + resolved "https://registry.npmmirror.com/@graphql-tools/merge/-/merge-8.2.14.tgz#44811e5453f5515d9537bd1b64f0f0cfe6313a45" + integrity sha512-od6lTF732nwPX91G79eiJf+dyRBHxCaKe7QL4IYeH4d1k+NYqx/ihYpFJNjDaqxmpHH92Hr+TxsP9SYRK3/QKg== + dependencies: + "@graphql-tools/utils" "8.6.13" + tslib "^2.4.0" + +"@graphql-tools/merge@^6.2.12": + version "6.2.17" + resolved "https://registry.npmmirror.com/@graphql-tools/merge/-/merge-6.2.17.tgz#4dedf87d8435a5e1091d7cc8d4f371ed1e029f1f" + integrity sha512-G5YrOew39fZf16VIrc49q3c8dBqQDD0ax5LYPiNja00xsXDi0T9zsEWVt06ApjtSdSF6HDddlu5S12QjeN8Tow== + dependencies: + "@graphql-tools/schema" "^8.0.2" + "@graphql-tools/utils" "8.0.2" + tslib "~2.3.0" + +"@graphql-tools/optimize@^1.0.1": + version "1.2.1" + resolved "https://registry.npmmirror.com/@graphql-tools/optimize/-/optimize-1.2.1.tgz#c1d26cb082877325f70911981cb703a0a805bda8" + integrity sha512-OAg1TYtYfeQMYlfsxNaY0FbEG4xsjdOHZw7/KFT1BdoCDtvl2NlYKoxh97mgZk2XmjqZULw/PS2E1MOk6IQapw== + dependencies: + tslib "^2.4.0" + +"@graphql-tools/relay-operation-optimizer@^6.3.7", "@graphql-tools/relay-operation-optimizer@^6.4.14": + version "6.4.14" + resolved "https://registry.npmmirror.com/@graphql-tools/relay-operation-optimizer/-/relay-operation-optimizer-6.4.14.tgz#812f32123a1351615a72b4e832852daac7bd64b3" + integrity sha512-vqch2M/sIUfMmlRJ4eCupiHlVPXWOPVpHX9yCZwRrpNg0Eaokyc57NSjJuKVV3KcvcJKf03qfMK2PxFbfDvN9A== + dependencies: + "@ardatan/relay-compiler" "12.0.0" + "@graphql-tools/utils" "8.6.13" + tslib "^2.4.0" + +"@graphql-tools/schema@8.3.14", "@graphql-tools/schema@^8.0.2", "@graphql-tools/schema@^8.1.2": + version "8.3.14" + resolved "https://registry.npmmirror.com/@graphql-tools/schema/-/schema-8.3.14.tgz#0aeab46daab70fb7505c950dc7e83a3da0eeb7ce" + integrity sha512-ntA4pKwyyPHFFKcIw17FfqGZAiTNZl0tHieQpPIkN5fPc4oHcXOfaj1vBjtIC/Qn6H7XBBu3l2kMA8FpobdxTQ== + dependencies: + "@graphql-tools/merge" "8.2.14" + "@graphql-tools/utils" "8.6.13" + tslib "^2.4.0" + value-or-promise "1.0.11" + +"@graphql-tools/schema@^7.0.0", "@graphql-tools/schema@^7.1.5": + version "7.1.5" + resolved "https://registry.npmmirror.com/@graphql-tools/schema/-/schema-7.1.5.tgz#07b24e52b182e736a6b77c829fc48b84d89aa711" + integrity sha512-uyn3HSNSckf4mvQSq0Q07CPaVZMNFCYEVxroApOaw802m9DcZPgf9XVPy/gda5GWj9AhbijfRYVTZQgHnJ4CXA== + dependencies: + "@graphql-tools/utils" "^7.1.2" + tslib "~2.2.0" + value-or-promise "1.0.6" + +"@graphql-tools/url-loader@^6.0.0": + version "6.10.1" + resolved "https://registry.npmmirror.com/@graphql-tools/url-loader/-/url-loader-6.10.1.tgz#dc741e4299e0e7ddf435eba50a1f713b3e763b33" + integrity sha512-DSDrbhQIv7fheQ60pfDpGD256ixUQIR6Hhf9Z5bRjVkXOCvO5XrkwoWLiU7iHL81GB1r0Ba31bf+sl+D4nyyfw== + dependencies: + "@graphql-tools/delegate" "^7.0.1" + "@graphql-tools/utils" "^7.9.0" + "@graphql-tools/wrap" "^7.0.4" + "@microsoft/fetch-event-source" "2.0.1" + "@types/websocket" "1.0.2" + abort-controller "3.0.0" + cross-fetch "3.1.4" + extract-files "9.0.0" + form-data "4.0.0" + graphql-ws "^4.4.1" + is-promise "4.0.0" + isomorphic-ws "4.0.1" + lodash "4.17.21" + meros "1.1.4" + subscriptions-transport-ws "^0.9.18" + sync-fetch "0.3.0" + tslib "~2.2.0" + valid-url "1.0.9" + ws "7.4.5" + +"@graphql-tools/utils@8.0.2": + version "8.0.2" + resolved "https://registry.npmmirror.com/@graphql-tools/utils/-/utils-8.0.2.tgz#795a8383cdfdc89855707d62491c576f439f3c51" + integrity sha512-gzkavMOgbhnwkHJYg32Adv6f+LxjbQmmbdD5Hty0+CWxvaiuJq+nU6tzb/7VSU4cwhbNLx/lGu2jbCPEW1McZQ== + dependencies: + tslib "~2.3.0" + +"@graphql-tools/utils@8.6.13", "@graphql-tools/utils@^8.1.1", "@graphql-tools/utils@^8.3.0", "@graphql-tools/utils@^8.5.2": + version "8.6.13" + resolved "https://registry.npmmirror.com/@graphql-tools/utils/-/utils-8.6.13.tgz#2b4fb7f9f8a29b25eecd44551fb95974de32f969" + integrity sha512-FiVqrQzj4cgz0HcZ3CxUs8NtBGPZFpmsVyIgwmL6YCwIhjJQnT72h8G3/vk5zVfjfesht85YGp0inWWuoCKWzg== + dependencies: + tslib "^2.4.0" + +"@graphql-tools/utils@^7.0.0", "@graphql-tools/utils@^7.1.2", "@graphql-tools/utils@^7.5.0", "@graphql-tools/utils@^7.7.0", "@graphql-tools/utils@^7.7.1", "@graphql-tools/utils@^7.8.1", "@graphql-tools/utils@^7.9.0": + version "7.10.0" + resolved "https://registry.npmmirror.com/@graphql-tools/utils/-/utils-7.10.0.tgz#07a4cb5d1bec1ff1dc1d47a935919ee6abd38699" + integrity sha512-d334r6bo9mxdSqZW6zWboEnnOOFRrAPVQJ7LkU8/6grglrbcu6WhwCLzHb90E94JI3TD3ricC3YGbUqIi9Xg0w== + dependencies: + "@ardatan/aggregate-error" "0.0.6" + camel-case "4.1.2" + tslib "~2.2.0" + +"@graphql-tools/wrap@^7.0.4": + version "7.0.8" + resolved "https://registry.npmmirror.com/@graphql-tools/wrap/-/wrap-7.0.8.tgz#ad41e487135ca3ea1ae0ea04bb3f596177fb4f50" + integrity sha512-1NDUymworsOlb53Qfh7fonDi2STvqCtbeE68ntKY9K/Ju/be2ZNxrFSbrBHwnxWcN9PjISNnLcAyJ1L5tCUyhg== + dependencies: + "@graphql-tools/delegate" "^7.1.5" + "@graphql-tools/schema" "^7.1.5" + "@graphql-tools/utils" "^7.8.1" + tslib "~2.2.0" + value-or-promise "1.0.6" + +"@hapi/address@2.x.x": + version "2.1.4" + resolved "https://registry.npmmirror.com/@hapi/address/-/address-2.1.4.tgz#5d67ed43f3fd41a69d4b9ff7b56e7c0d1d0a81e5" + integrity sha512-QD1PhQk+s31P1ixsX0H0Suoupp3VMXzIVMSwobR3F3MSUO2YCV0B7xqLcUw/Bh8yuvd3LhpyqLQWTNcRmp6IdQ== + +"@hapi/bourne@1.x.x": + version "1.3.2" + resolved "https://registry.npmmirror.com/@hapi/bourne/-/bourne-1.3.2.tgz#0a7095adea067243ce3283e1b56b8a8f453b242a" + integrity sha512-1dVNHT76Uu5N3eJNTYcvxee+jzX4Z9lfciqRRHCU27ihbUcYi+iSc2iml5Ke1LXe1SyJCLA0+14Jh4tXJgOppA== + +"@hapi/hoek@8.x.x", "@hapi/hoek@^8.3.0": + version "8.5.1" + resolved "https://registry.npmmirror.com/@hapi/hoek/-/hoek-8.5.1.tgz#fde96064ca446dec8c55a8c2f130957b070c6e06" + integrity sha512-yN7kbciD87WzLGc5539Tn0sApjyiGHAJgKvG9W8C7O+6c7qmoQMfVs0W4bX17eqz6C78QJqqFrtgdK5EWf6Qow== + +"@hapi/hoek@^9.0.0": + version "9.3.0" + resolved "https://registry.npmmirror.com/@hapi/hoek/-/hoek-9.3.0.tgz#8368869dcb735be2e7f5cb7647de78e167a251fb" + integrity sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ== + +"@hapi/joi@^15.0.0": + version "15.1.1" + resolved "https://registry.npmmirror.com/@hapi/joi/-/joi-15.1.1.tgz#c675b8a71296f02833f8d6d243b34c57b8ce19d7" + integrity sha512-entf8ZMOK8sc+8YfeOlM8pCfg3b5+WZIKBfUaaJT8UsjAAPjartzxIYm3TIbjvA4u+u++KbcXD38k682nVHDAQ== + dependencies: + "@hapi/address" "2.x.x" + "@hapi/bourne" "1.x.x" + "@hapi/hoek" "8.x.x" + "@hapi/topo" "3.x.x" + +"@hapi/topo@3.x.x": + version "3.1.6" + resolved "https://registry.npmmirror.com/@hapi/topo/-/topo-3.1.6.tgz#68d935fa3eae7fdd5ab0d7f953f3205d8b2bfc29" + integrity sha512-tAag0jEcjwH+P2quUfipd7liWCNX2F8NvYjQp2wtInsZxnMlypdw0FtAOLxtvvkO+GSRRbmNi8m/5y42PQJYCQ== + dependencies: + "@hapi/hoek" "^8.3.0" + +"@hapi/topo@^5.0.0": + version "5.1.0" + resolved "https://registry.npmmirror.com/@hapi/topo/-/topo-5.1.0.tgz#dc448e332c6c6e37a4dc02fd84ba8d44b9afb012" + integrity sha512-foQZKJig7Ob0BMAYBfcJk8d77QtOe7Wo4ox7ff1lQYoNNAb6jwcY1ncdoy2e9wQZzvNy7ODZCYJkK8kzmcAnAg== + dependencies: + "@hapi/hoek" "^9.0.0" + +"@humanwhocodes/config-array@^0.5.0": + version "0.5.0" + resolved "https://registry.npmmirror.com/@humanwhocodes/config-array/-/config-array-0.5.0.tgz#1407967d4c6eecd7388f83acf1eaf4d0c6e58ef9" + integrity sha512-FagtKFz74XrTl7y6HCzQpwDfXP0yhxe9lHLD1UZxjvZIcbyRz8zTFF/yYNfSfzU414eDwZ1SrO0Qvtyf+wFMQg== + dependencies: + "@humanwhocodes/object-schema" "^1.2.0" + debug "^4.1.1" + minimatch "^3.0.4" + +"@humanwhocodes/object-schema@^1.2.0": + version "1.2.1" + resolved "https://registry.npmmirror.com/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz#b520529ec21d8e5945a1851dfd1c32e94e39ff45" + integrity sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA== + +"@iarna/toml@^2.2.5": + version "2.2.5" + resolved "https://registry.npmmirror.com/@iarna/toml/-/toml-2.2.5.tgz#b32366c89b43c6f8cefbdefac778b9c828e3ba8c" + integrity sha512-trnsAYxU3xnS1gPHPyU961coFyLkh4gAD/0zQ5mymY4yOZ+CYvsPqUbOFSw0aDM4y0tV7tiFxL/1XfXPNC6IPg== + +"@icon-park/react@^1.3.5": + version "1.4.0" + resolved "https://registry.npmmirror.com/@icon-park/react/-/react-1.4.0.tgz#f9dfd6eaa8525cf4e8ad24e1043e8a1ba5857d5a" + integrity sha512-2XKUKW5ChK4tTJNMin6Rzq/VSBUg2Os72CGequapJV+6BYz2ytigUtZxxmHngYfUx3cnzLdHwyxwvkjolQHJwQ== + +"@jimp/bmp@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/bmp/-/bmp-0.16.1.tgz#6e2da655b2ba22e721df0795423f34e92ef13768" + integrity sha512-iwyNYQeBawrdg/f24x3pQ5rEx+/GwjZcCXd3Kgc+ZUd+Ivia7sIqBsOnDaMZdKCBPlfW364ekexnlOqyVa0NWg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + bmp-js "^0.1.0" + +"@jimp/core@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/core/-/core-0.16.1.tgz#68c4288f6ef7f31a0f6b859ba3fb28dae930d39d" + integrity sha512-la7kQia31V6kQ4q1kI/uLimu8FXx7imWVajDGtwUG8fzePLWDFJyZl0fdIXVCL1JW2nBcRHidUot6jvlRDi2+g== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + any-base "^1.1.0" + buffer "^5.2.0" + exif-parser "^0.1.12" + file-type "^9.0.0" + load-bmfont "^1.3.1" + mkdirp "^0.5.1" + phin "^2.9.1" + pixelmatch "^4.0.2" + tinycolor2 "^1.4.1" + +"@jimp/custom@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/custom/-/custom-0.16.1.tgz#28b659c59e20a1d75a0c46067bd3f4bd302cf9c5" + integrity sha512-DNUAHNSiUI/j9hmbatD6WN/EBIyeq4AO0frl5ETtt51VN1SvE4t4v83ZA/V6ikxEf3hxLju4tQ5Pc3zmZkN/3A== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/core" "^0.16.1" + +"@jimp/gif@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/gif/-/gif-0.16.1.tgz#d1f7c3a58f4666482750933af8b8f4666414f3ca" + integrity sha512-r/1+GzIW1D5zrP4tNrfW+3y4vqD935WBXSc8X/wm23QTY9aJO9Lw6PEdzpYCEY+SOklIFKaJYUAq/Nvgm/9ryw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + gifwrap "^0.9.2" + omggif "^1.0.9" + +"@jimp/jpeg@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/jpeg/-/jpeg-0.16.1.tgz#3b7bb08a4173f2f6d81f3049b251df3ee2ac8175" + integrity sha512-8352zrdlCCLFdZ/J+JjBslDvml+fS3Z8gttdml0We759PnnZGqrnPRhkOEOJbNUlE+dD4ckLeIe6NPxlS/7U+w== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + jpeg-js "0.4.2" + +"@jimp/plugin-blit@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-blit/-/plugin-blit-0.16.1.tgz#09ea919f9d326de3b9c2826fe4155da37dde8edb" + integrity sha512-fKFNARm32RoLSokJ8WZXHHH2CGzz6ire2n1Jh6u+XQLhk9TweT1DcLHIXwQMh8oR12KgjbgsMGvrMVlVknmOAg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-blur@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-blur/-/plugin-blur-0.16.1.tgz#e614fa002797dcd662e705d4cea376e7db968bf5" + integrity sha512-1WhuLGGj9MypFKRcPvmW45ht7nXkOKu+lg3n2VBzIB7r4kKNVchuI59bXaCYQumOLEqVK7JdB4glaDAbCQCLyw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-circle@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-circle/-/plugin-circle-0.16.1.tgz#20e3194a67ca29740aba2630fd4d0a89afa27491" + integrity sha512-JK7yi1CIU7/XL8hdahjcbGA3V7c+F+Iw+mhMQhLEi7Q0tCnZ69YJBTamMiNg3fWPVfMuvWJJKOBRVpwNTuaZRg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-color@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-color/-/plugin-color-0.16.1.tgz#0f298ba74dee818b663834cd80d53e56f3755233" + integrity sha512-9yQttBAO5SEFj7S6nJK54f+1BnuBG4c28q+iyzm1JjtnehjqMg6Ljw4gCSDCvoCQ3jBSYHN66pmwTV74SU1B7A== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + tinycolor2 "^1.4.1" + +"@jimp/plugin-contain@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-contain/-/plugin-contain-0.16.1.tgz#3c5f5c495fd9bb08a970739d83694934f58123f2" + integrity sha512-44F3dUIjBDHN+Ym/vEfg+jtjMjAqd2uw9nssN67/n4FdpuZUVs7E7wadKY1RRNuJO+WgcD5aDQcsvurXMETQTg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-cover@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-cover/-/plugin-cover-0.16.1.tgz#0e8caec16a40abe15b1b32e5383a603a3306dc41" + integrity sha512-YztWCIldBAVo0zxcQXR+a/uk3/TtYnpKU2CanOPJ7baIuDlWPsG+YE4xTsswZZc12H9Kl7CiziEbDtvF9kwA/Q== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-crop@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-crop/-/plugin-crop-0.16.1.tgz#b362497c873043fe47ba881ab08604bf7226f50f" + integrity sha512-UQdva9oQzCVadkyo3T5Tv2CUZbf0klm2cD4cWMlASuTOYgaGaFHhT9st+kmfvXjKL8q3STkBu/zUPV6PbuV3ew== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-displace@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-displace/-/plugin-displace-0.16.1.tgz#4dd9db518c3e78de9d723f86a234bf98922afe8d" + integrity sha512-iVAWuz2+G6Heu8gVZksUz+4hQYpR4R0R/RtBzpWEl8ItBe7O6QjORAkhxzg+WdYLL2A/Yd4ekTpvK0/qW8hTVw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-dither@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-dither/-/plugin-dither-0.16.1.tgz#b47de2c0bb09608bed228b41c3cd01a85ec2d45b" + integrity sha512-tADKVd+HDC9EhJRUDwMvzBXPz4GLoU6s5P7xkVq46tskExYSptgj5713J5Thj3NMgH9Rsqu22jNg1H/7tr3V9Q== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-fisheye@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-fisheye/-/plugin-fisheye-0.16.1.tgz#f625047b6cdbe1b83b89e9030fd025ab19cdb1a4" + integrity sha512-BWHnc5hVobviTyIRHhIy9VxI1ACf4CeSuCfURB6JZm87YuyvgQh5aX5UDKtOz/3haMHXBLP61ZBxlNpMD8CG4A== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-flip@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-flip/-/plugin-flip-0.16.1.tgz#7a99ea22bde802641017ed0f2615870c144329bb" + integrity sha512-KdxTf0zErfZ8DyHkImDTnQBuHby+a5YFdoKI/G3GpBl3qxLBvC+PWkS2F/iN3H7wszP7/TKxTEvWL927pypT0w== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-gaussian@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-gaussian/-/plugin-gaussian-0.16.1.tgz#0845e314085ccd52e34fad9a83949bc0d81a68e8" + integrity sha512-u9n4wjskh3N1mSqketbL6tVcLU2S5TEaFPR40K6TDv4phPLZALi1Of7reUmYpVm8mBDHt1I6kGhuCJiWvzfGyg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-invert@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-invert/-/plugin-invert-0.16.1.tgz#7e6f5a15707256f3778d06921675bbcf18545c97" + integrity sha512-2DKuyVXANH8WDpW9NG+PYFbehzJfweZszFYyxcaewaPLN0GxvxVLOGOPP1NuUTcHkOdMFbE0nHDuB7f+sYF/2w== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-mask@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-mask/-/plugin-mask-0.16.1.tgz#e7f2460e05c3cda7af5e76f33ccb0579f66f90df" + integrity sha512-snfiqHlVuj4bSFS0v96vo2PpqCDMe4JB+O++sMo5jF5mvGcGL6AIeLo8cYqPNpdO6BZpBJ8MY5El0Veckhr39Q== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-normalize@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-normalize/-/plugin-normalize-0.16.1.tgz#032dfd88eefbc4dedc8b1b2d243832e4f3af30c8" + integrity sha512-dOQfIOvGLKDKXPU8xXWzaUeB0nvkosHw6Xg1WhS1Z5Q0PazByhaxOQkSKgUryNN/H+X7UdbDvlyh/yHf3ITRaw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-print@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-print/-/plugin-print-0.16.1.tgz#66b803563f9d109825970714466e6ab9ae639ff6" + integrity sha512-ceWgYN40jbN4cWRxixym+csyVymvrryuKBQ+zoIvN5iE6OyS+2d7Mn4zlNgumSczb9GGyZZESIgVcBDA1ezq0Q== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + load-bmfont "^1.4.0" + +"@jimp/plugin-resize@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-resize/-/plugin-resize-0.16.1.tgz#65e39d848ed13ba2d6c6faf81d5d590396571d10" + integrity sha512-u4JBLdRI7dargC04p2Ha24kofQBk3vhaf0q8FwSYgnCRwxfvh2RxvhJZk9H7Q91JZp6wgjz/SjvEAYjGCEgAwQ== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-rotate@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-rotate/-/plugin-rotate-0.16.1.tgz#53fb5d51a4b3d05af9c91c2a8fffe5d7a1a47c8c" + integrity sha512-ZUU415gDQ0VjYutmVgAYYxC9Og9ixu2jAGMCU54mSMfuIlmohYfwARQmI7h4QB84M76c9hVLdONWjuo+rip/zg== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-scale@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-scale/-/plugin-scale-0.16.1.tgz#89f6ba59feed3429847ed226aebda33a240cc647" + integrity sha512-jM2QlgThIDIc4rcyughD5O7sOYezxdafg/2Xtd1csfK3z6fba3asxDwthqPZAgitrLgiKBDp6XfzC07Y/CefUw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-shadow@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-shadow/-/plugin-shadow-0.16.1.tgz#a7af892a740febf41211e10a5467c3c5c521a04c" + integrity sha512-MeD2Is17oKzXLnsphAa1sDstTu6nxscugxAEk3ji0GV1FohCvpHBcec0nAq6/czg4WzqfDts+fcPfC79qWmqrA== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugin-threshold@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugin-threshold/-/plugin-threshold-0.16.1.tgz#34f3078f9965145b7ae26c53a32ad74b1195bbf5" + integrity sha512-iGW8U/wiCSR0+6syrPioVGoSzQFt4Z91SsCRbgNKTAk7D+XQv6OI78jvvYg4o0c2FOlwGhqz147HZV5utoSLxA== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + +"@jimp/plugins@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/plugins/-/plugins-0.16.1.tgz#9f08544c97226d6460a16ced79f57e85bec3257b" + integrity sha512-c+lCqa25b+4q6mJZSetlxhMoYuiltyS+ValLzdwK/47+aYsq+kcJNl+TuxIEKf59yr9+5rkbpsPkZHLF/V7FFA== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/plugin-blit" "^0.16.1" + "@jimp/plugin-blur" "^0.16.1" + "@jimp/plugin-circle" "^0.16.1" + "@jimp/plugin-color" "^0.16.1" + "@jimp/plugin-contain" "^0.16.1" + "@jimp/plugin-cover" "^0.16.1" + "@jimp/plugin-crop" "^0.16.1" + "@jimp/plugin-displace" "^0.16.1" + "@jimp/plugin-dither" "^0.16.1" + "@jimp/plugin-fisheye" "^0.16.1" + "@jimp/plugin-flip" "^0.16.1" + "@jimp/plugin-gaussian" "^0.16.1" + "@jimp/plugin-invert" "^0.16.1" + "@jimp/plugin-mask" "^0.16.1" + "@jimp/plugin-normalize" "^0.16.1" + "@jimp/plugin-print" "^0.16.1" + "@jimp/plugin-resize" "^0.16.1" + "@jimp/plugin-rotate" "^0.16.1" + "@jimp/plugin-scale" "^0.16.1" + "@jimp/plugin-shadow" "^0.16.1" + "@jimp/plugin-threshold" "^0.16.1" + timm "^1.6.1" + +"@jimp/png@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/png/-/png-0.16.1.tgz#f24cfc31529900b13a2dd9d4fdb4460c1e4d814e" + integrity sha512-iyWoCxEBTW0OUWWn6SveD4LePW89kO7ZOy5sCfYeDM/oTPLpR8iMIGvZpZUz1b8kvzFr27vPst4E5rJhGjwsdw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/utils" "^0.16.1" + pngjs "^3.3.3" + +"@jimp/tiff@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/tiff/-/tiff-0.16.1.tgz#0e8756695687d7574b6bc73efab0acd4260b7a12" + integrity sha512-3K3+xpJS79RmSkAvFMgqY5dhSB+/sxhwTFA9f4AVHUK0oKW+u6r52Z1L0tMXHnpbAdR9EJ+xaAl2D4x19XShkQ== + dependencies: + "@babel/runtime" "^7.7.2" + utif "^2.0.1" + +"@jimp/types@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/types/-/types-0.16.1.tgz#0dbab37b3202315c91010f16c31766d35a2322cc" + integrity sha512-g1w/+NfWqiVW4CaXSJyD28JQqZtm2eyKMWPhBBDCJN9nLCN12/Az0WFF3JUAktzdsEC2KRN2AqB1a2oMZBNgSQ== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/bmp" "^0.16.1" + "@jimp/gif" "^0.16.1" + "@jimp/jpeg" "^0.16.1" + "@jimp/png" "^0.16.1" + "@jimp/tiff" "^0.16.1" + timm "^1.6.1" + +"@jimp/utils@^0.16.1": + version "0.16.1" + resolved "https://registry.npmmirror.com/@jimp/utils/-/utils-0.16.1.tgz#2f51e6f14ff8307c4aa83d5e1a277da14a9fe3f7" + integrity sha512-8fULQjB0x4LzUSiSYG6ZtQl355sZjxbv8r9PPAuYHzS9sGiSHJQavNqK/nKnpDsVkU88/vRGcE7t3nMU0dEnVw== + dependencies: + "@babel/runtime" "^7.7.2" + regenerator-runtime "^0.13.3" + +"@jridgewell/gen-mapping@^0.1.0": + version "0.1.1" + resolved "https://registry.npmmirror.com/@jridgewell/gen-mapping/-/gen-mapping-0.1.1.tgz#e5d2e450306a9491e3bd77e323e38d7aff315996" + integrity sha512-sQXCasFk+U8lWYEe66WxRDOE9PjVz4vSM51fTu3Hw+ClTpUSQb718772vH3pyS5pShp6lvQM7SxgIDXXXmOX7w== + dependencies: + "@jridgewell/set-array" "^1.0.0" + "@jridgewell/sourcemap-codec" "^1.4.10" + +"@jridgewell/gen-mapping@^0.3.0": + version "0.3.1" + resolved "https://registry.npmmirror.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.1.tgz#cf92a983c83466b8c0ce9124fadeaf09f7c66ea9" + integrity sha512-GcHwniMlA2z+WFPWuY8lp3fsza0I8xPFMWL5+n8LYyP6PSvPrXf4+n8stDHZY2DM0zy9sVkRDy1jDI4XGzYVqg== + dependencies: + "@jridgewell/set-array" "^1.0.0" + "@jridgewell/sourcemap-codec" "^1.4.10" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/resolve-uri@^3.0.3": + version "3.0.7" + resolved "https://registry.npmmirror.com/@jridgewell/resolve-uri/-/resolve-uri-3.0.7.tgz#30cd49820a962aff48c8fffc5cd760151fca61fe" + integrity sha512-8cXDaBBHOr2pQ7j77Y6Vp5VDT2sIqWyWQ56TjEq4ih/a4iST3dItRe8Q9fp0rrIl9DoKhWQtUQz/YpOxLkXbNA== + +"@jridgewell/set-array@^1.0.0": + version "1.1.1" + resolved "https://registry.npmmirror.com/@jridgewell/set-array/-/set-array-1.1.1.tgz#36a6acc93987adcf0ba50c66908bd0b70de8afea" + integrity sha512-Ct5MqZkLGEXTVmQYbGtx9SVqD2fqwvdubdps5D3djjAkgkKwT918VNOz65pEHFaYTeWcukmJmH5SwsA9Tn2ObQ== + +"@jridgewell/source-map@^0.3.2": + version "0.3.2" + resolved "https://registry.npmmirror.com/@jridgewell/source-map/-/source-map-0.3.2.tgz#f45351aaed4527a298512ec72f81040c998580fb" + integrity sha512-m7O9o2uR8k2ObDysZYzdfhb08VuEml5oWGiosa1VdaPZ/A6QyPkAJuwN0Q1lhULOf6B7MtQmHENS743hWtCrgw== + dependencies: + "@jridgewell/gen-mapping" "^0.3.0" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/sourcemap-codec@^1.4.10": + version "1.4.13" + resolved "https://registry.npmmirror.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.13.tgz#b6461fb0c2964356c469e115f504c95ad97ab88c" + integrity sha512-GryiOJmNcWbovBxTfZSF71V/mXbgcV3MewDe3kIMCLyIh5e7SKAeUZs+rMnJ8jkMolZ/4/VsdBmMrw3l+VdZ3w== + +"@jridgewell/trace-mapping@^0.3.13", "@jridgewell/trace-mapping@^0.3.7", "@jridgewell/trace-mapping@^0.3.9": + version "0.3.13" + resolved "https://registry.npmmirror.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.13.tgz#dcfe3e95f224c8fe97a87a5235defec999aa92ea" + integrity sha512-o1xbKhp9qnIAoHJSWd6KlCZfqslL4valSF81H8ImioOAxluWYWOpWkpyktY2vnt4tbrX9XYaxovq6cgowaJp2w== + dependencies: + "@jridgewell/resolve-uri" "^3.0.3" + "@jridgewell/sourcemap-codec" "^1.4.10" + +"@lezer/common@^0.15.0", "@lezer/common@^0.15.7": + version "0.15.12" + resolved "https://registry.npmmirror.com/@lezer/common/-/common-0.15.12.tgz#2f21aec551dd5fd7d24eb069f90f54d5bc6ee5e9" + integrity sha512-edfwCxNLnzq5pBA/yaIhwJ3U3Kz8VAUOTRg0hhxaizaI1N+qxV7EXDv/kLCkLeq2RzSFvxexlaj5Mzfn2kY0Ig== + +"@lezer/lr@^0.15.4": + version "0.15.8" + resolved "https://registry.npmmirror.com/@lezer/lr/-/lr-0.15.8.tgz#1564a911e62b0a0f75ca63794a6aa8c5dc63db21" + integrity sha512-bM6oE6VQZ6hIFxDNKk8bKPa14hqFrV07J/vHGOeiAbJReIaQXmkVb6xQu4MR+JBTLa5arGRyAAjJe1qaQt3Uvg== + dependencies: + "@lezer/common" "^0.15.0" + +"@lmdb/lmdb-darwin-arm64@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-darwin-arm64/-/lmdb-darwin-arm64-2.5.2.tgz#bc66fa43286b5c082e8fee0eacc17995806b6fbe" + integrity sha512-+F8ioQIUN68B4UFiIBYu0QQvgb9FmlKw2ctQMSBfW2QBrZIxz9vD9jCGqTCPqZBRbPHAS/vG1zSXnKqnS2ch/A== + +"@lmdb/lmdb-darwin-x64@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-darwin-x64/-/lmdb-darwin-x64-2.5.2.tgz#89d8390041bce6bab24a82a20392be22faf54ffc" + integrity sha512-KvPH56KRLLx4KSfKBx0m1r7GGGUMXm0jrKmNE7plbHlesZMuPJICtn07HYgQhj1LNsK7Yqwuvnqh1QxhJnF1EA== + +"@lmdb/lmdb-linux-arm64@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-linux-arm64/-/lmdb-linux-arm64-2.5.2.tgz#14fe4c96c2bb1285f93797f45915fa35ee047268" + integrity sha512-aLl89VHL/wjhievEOlPocoefUyWdvzVrcQ/MHQYZm2JfV1jUsrbr/ZfkPPUFvZBf+VSE+Q0clWs9l29PCX1hTQ== + +"@lmdb/lmdb-linux-arm@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-linux-arm/-/lmdb-linux-arm-2.5.2.tgz#05bde4573ab10cf21827339fe687148f2590cfa1" + integrity sha512-5kQAP21hAkfW5Bl+e0P57dV4dGYnkNIpR7f/GAh6QHlgXx+vp/teVj4PGRZaKAvt0GX6++N6hF8NnGElLDuIDw== + +"@lmdb/lmdb-linux-x64@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-linux-x64/-/lmdb-linux-x64-2.5.2.tgz#d2f85afd857d2c33d2caa5b057944574edafcfee" + integrity sha512-xUdUfwDJLGjOUPH3BuPBt0NlIrR7f/QHKgu3GZIXswMMIihAekj2i97oI0iWG5Bok/b+OBjHPfa8IU9velnP/Q== + +"@lmdb/lmdb-win32-x64@2.5.2": + version "2.5.2" + resolved "https://registry.npmmirror.com/@lmdb/lmdb-win32-x64/-/lmdb-win32-x64-2.5.2.tgz#28f643fbc0bec30b07fbe95b137879b6b4d1c9c5" + integrity sha512-zrBczSbXKxEyK2ijtbRdICDygRqWSRPpZMN5dD1T8VMEW5RIhIbwFWw2phDRXuBQdVDpSjalCIUMWMV2h3JaZA== + +"@mantine/core@^4.2.5": + version "4.2.9" + resolved "https://registry.npmmirror.com/@mantine/core/-/core-4.2.9.tgz#395895457a8b616abdb37f16c896e243281cc5fe" + integrity sha512-25krgKF7FUFDsF5IYk+A/7GjgAZFzEcaFmGvscXS/7ccOickBmXzENwY9eJpv+MDo/7Sj5eiS0ZBi+WN27tdcw== + dependencies: + "@mantine/styles" "4.2.9" + "@popperjs/core" "^2.9.3" + "@radix-ui/react-scroll-area" "^0.1.1" + react-popper "^2.2.5" + react-textarea-autosize "^8.3.2" + +"@mantine/hooks@^4.2.5": + version "4.2.9" + resolved "https://registry.npmmirror.com/@mantine/hooks/-/hooks-4.2.9.tgz#ada020b0a2820dc208e8dde23374e2f260d9111c" + integrity sha512-oLtlSXkl99vPGBTR9yoVCWoCQ25Bc4WhPci31iERMLMM/0fz85CZayQcAC4etq6un6rDiRNu6jq0YB1S5stmPg== + +"@mantine/ssr@>=3.6.14": + version "4.2.9" + resolved "https://registry.npmmirror.com/@mantine/ssr/-/ssr-4.2.9.tgz#4ce7c687dfa12e4aab46ff2e78871fde0cd749d8" + integrity sha512-6B5R8ELRR+GQaXdnUUoSlZXEA5DiG4Pp91EOJTU+UYnieQ+gdLISp2HmYAvKMpETViOciBou6r8j2hH74tls4g== + dependencies: + "@emotion/cache" "11.7.1" + "@emotion/react" "11.7.1" + "@emotion/serialize" "1.0.2" + "@emotion/server" "11.4.0" + "@emotion/utils" "1.0.0" + "@mantine/styles" "4.2.9" + csstype "3.0.9" + html-react-parser "1.3.0" + +"@mantine/styles@4.2.9": + version "4.2.9" + resolved "https://registry.npmmirror.com/@mantine/styles/-/styles-4.2.9.tgz#97fbc96be5ea970218aab0ad62a8f7c736bc6225" + integrity sha512-UT5m1412N20HPl+Mg2zTN7gP34J1GIa3o7eQ+fHThAQkAMm/vBehzX8nkbdxwyYkwuu6+/4kizcbM/Op4jMI3g== + dependencies: + "@emotion/cache" "11.7.1" + "@emotion/react" "11.7.1" + "@emotion/serialize" "1.0.2" + "@emotion/utils" "1.0.0" + clsx "^1.1.1" + csstype "3.0.9" + +"@microsoft/fetch-event-source@2.0.1": + version "2.0.1" + resolved "https://registry.npmmirror.com/@microsoft/fetch-event-source/-/fetch-event-source-2.0.1.tgz#9ceecc94b49fbaa15666e38ae8587f64acce007d" + integrity sha512-W6CLUJ2eBMw3Rec70qrsEW0jOm/3twwJv21mrmj2yORiaVmVYGS4sSS5yUwvQc1ZlDLYGPnClVWmUUMagKNsfA== + +"@mischnic/json-sourcemap@^0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@mischnic/json-sourcemap/-/json-sourcemap-0.1.0.tgz#38af657be4108140a548638267d02a2ea3336507" + integrity sha512-dQb3QnfNqmQNYA4nFSN/uLaByIic58gOXq4Y4XqLOWmOrw73KmJPt/HLyG0wvn1bnR6mBKs/Uwvkh+Hns1T0XA== + dependencies: + "@lezer/common" "^0.15.7" + "@lezer/lr" "^0.15.4" + json5 "^2.2.1" + +"@msgpackr-extract/msgpackr-extract-darwin-arm64@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-darwin-arm64/-/msgpackr-extract-darwin-arm64-2.0.2.tgz#01e3669b8b2dc01f6353f2c87e1ec94faf52c587" + integrity sha512-FMX5i7a+ojIguHpWbzh5MCsCouJkwf4z4ejdUY/fsgB9Vkdak4ZnoIEskOyOUMMB4lctiZFGszFQJXUeFL8tRg== + +"@msgpackr-extract/msgpackr-extract-darwin-x64@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-darwin-x64/-/msgpackr-extract-darwin-x64-2.0.2.tgz#5ca32f16e6f1b7854001a1a2345b61d4e26a0931" + integrity sha512-DznYtF3lHuZDSRaIOYeif4JgO0NtO2Xf8DsngAugMx/bUdTFbg86jDTmkVJBNmV+cxszz6OjGvinnS8AbJ342g== + +"@msgpackr-extract/msgpackr-extract-linux-arm64@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-linux-arm64/-/msgpackr-extract-linux-arm64-2.0.2.tgz#ff629f94379981bf476dffb1439a7c1d3dba2d72" + integrity sha512-b0jMEo566YdM2K+BurSed7bswjo3a6bcdw5ETqoIfSuxKuRLPfAiOjVbZyZBgx3J/TAM/QrvEQ/VN89A0ZAxSg== + +"@msgpackr-extract/msgpackr-extract-linux-arm@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-linux-arm/-/msgpackr-extract-linux-arm-2.0.2.tgz#5f6fd30d266c4a90cf989049c7f2e50e5d4fcd4c" + integrity sha512-Gy9+c3Wj+rUlD3YvCZTi92gs+cRX7ZQogtwq0IhRenloTTlsbpezNgk6OCkt59V4ATEWSic9rbU92H/l7XsRvA== + +"@msgpackr-extract/msgpackr-extract-linux-x64@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-linux-x64/-/msgpackr-extract-linux-x64-2.0.2.tgz#167faa553b9dbffac8b03bf27de9b6f846f0e1bc" + integrity sha512-zrBHaePwcv4cQXxzYgNj0+A8I1uVN97E7/3LmkRocYZ+rMwUsnPpp4RuTAHSRoKlTQV3nSdCQW4Qdt4MXw/iHw== + +"@msgpackr-extract/msgpackr-extract-win32-x64@2.0.2": + version "2.0.2" + resolved "https://registry.npmmirror.com/@msgpackr-extract/msgpackr-extract-win32-x64/-/msgpackr-extract-win32-x64-2.0.2.tgz#baea7764b1adf201ce4a792fe971fd7211dad2e4" + integrity sha512-fpnI00dt+yO1cKx9qBXelKhPBdEgvc8ZPav1+0r09j0woYQU2N79w/jcGawSY5UGlgQ3vjaJsFHnGbGvvqdLzg== + +"@nodelib/fs.scandir@2.1.5": + version "2.1.5" + resolved "https://registry.npmmirror.com/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz#7619c2eb21b25483f6d167548b4cfd5a7488c3d5" + integrity sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g== + dependencies: + "@nodelib/fs.stat" "2.0.5" + run-parallel "^1.1.9" + +"@nodelib/fs.stat@2.0.5", "@nodelib/fs.stat@^2.0.2": + version "2.0.5" + resolved "https://registry.npmmirror.com/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz#5bd262af94e9d25bd1e71b05deed44876a222e8b" + integrity sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A== + +"@nodelib/fs.walk@^1.2.3", "@nodelib/fs.walk@^1.2.8": + version "1.2.8" + resolved "https://registry.npmmirror.com/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz#e95737e8bb6746ddedf69c556953494f196fe69a" + integrity sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg== + dependencies: + "@nodelib/fs.scandir" "2.1.5" + fastq "^1.6.0" + +"@parcel/bundler-default@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/bundler-default/-/bundler-default-2.6.0.tgz#3b091d2f6ebdb333558fc25c1fb98e28b854ace3" + integrity sha512-AplEdGm/odV7yGmoeOnglxnY31WlNB5EqGLFGxkgs7uwDaTWoTX/9SWPG6xfvirhjDpms8sLSiVuBdFRCCLtNA== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/hash" "2.6.0" + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/cache@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/cache/-/cache-2.6.0.tgz#19a5132e5715d7ab1df7cb7a5ae5e8c29003a7b1" + integrity sha512-4vbD5uSuf+kRnrFesKhpn0AKnOw8u2UlvCyrplYmp1g9bNAkIooC/nDGdmkb/9SviPEbni9PEanQEHDU2+slpA== + dependencies: + "@parcel/fs" "2.6.0" + "@parcel/logger" "2.6.0" + "@parcel/utils" "2.6.0" + lmdb "2.3.10" + +"@parcel/codeframe@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/codeframe/-/codeframe-2.6.0.tgz#1de477a8772191d5990348b6c75922c1350b835c" + integrity sha512-yXXxrO9yyedHKpTwC+Af0+vPmQm+A9xeEhkt4f0yVg1n4t4yUIxYlTedzbM8ygZEEBtkXU9jJ+PkgXbfMf0dqw== + dependencies: + chalk "^4.1.0" + +"@parcel/compressor-raw@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/compressor-raw/-/compressor-raw-2.6.0.tgz#d8b238db719f43807ebd96ec08270b3c937221e2" + integrity sha512-rtMU2mGl88bic6Xbq1u5L49bMK4s5185b0k7h3JRdS6/0rR+Xp4k/o9Wog+hHjK/s82z1eF9WmET779ZpIDIQQ== + dependencies: + "@parcel/plugin" "2.6.0" + +"@parcel/core@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/core/-/core-2.6.0.tgz#dad1f5f529ffb47df772c155ef09119d3294538c" + integrity sha512-8OOWbPuxpFydpwNyKoz6d3e3O4DmxNYmMw4DXwrPSj/jyg7oa+SDtMT0/VXEhujE0HYkQPCHt4npRajkSuf99A== + dependencies: + "@mischnic/json-sourcemap" "^0.1.0" + "@parcel/cache" "2.6.0" + "@parcel/diagnostic" "2.6.0" + "@parcel/events" "2.6.0" + "@parcel/fs" "2.6.0" + "@parcel/graph" "2.6.0" + "@parcel/hash" "2.6.0" + "@parcel/logger" "2.6.0" + "@parcel/package-manager" "2.6.0" + "@parcel/plugin" "2.6.0" + "@parcel/source-map" "^2.0.0" + "@parcel/types" "2.6.0" + "@parcel/utils" "2.6.0" + "@parcel/workers" "2.6.0" + abortcontroller-polyfill "^1.1.9" + base-x "^3.0.8" + browserslist "^4.6.6" + clone "^2.1.1" + dotenv "^7.0.0" + dotenv-expand "^5.1.0" + json5 "^2.2.0" + msgpackr "^1.5.4" + nullthrows "^1.1.1" + semver "^5.7.1" + +"@parcel/diagnostic@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/diagnostic/-/diagnostic-2.6.0.tgz#99570b28ed44d64d57a3c3521bcfa4f6f631b495" + integrity sha512-+p8gC2FKxSI2veD7SoaNlP572v4kw+nafCQEPDtJuzYYRqywYUGncch25dkpgNApB4W4cXVkZu3ZbtIpCAmjQQ== + dependencies: + "@mischnic/json-sourcemap" "^0.1.0" + nullthrows "^1.1.1" + +"@parcel/events@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/events/-/events-2.6.0.tgz#6066c8c7b320e12fd206877bd549825b7eea8c63" + integrity sha512-2WaKtBs4iYwS88j4zRdyTJTgh8iuY4E32FMmjzzbheqETs6I05gWuPReGukJYxk8vc0Ir7tbzp12oAfpgo0Y+g== + +"@parcel/fs-search@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/fs-search/-/fs-search-2.6.0.tgz#35c52da3186cab953cf6686304921a7ab0c81be8" + integrity sha512-1nXzM3H/cA4kzLKvDBvwmNisKCdRqlgkLXh+OR1Zu28Kn4W34KuJMcHWW8cC+WIuuKqDh5oo2WPsC5y65GXBKQ== + dependencies: + detect-libc "^1.0.3" + +"@parcel/fs@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/fs/-/fs-2.6.0.tgz#287a3cda558f16aae5c67ccbe33a17c1bbd75ceb" + integrity sha512-6vxtx5Zy6MvDvH1EPx9JxjKGF03bR7VE1dUf4HLeX2D8YmpL5hkHJnlRCFdcH08rzOVwaJLzg1QNtblWJXQ9CA== + dependencies: + "@parcel/fs-search" "2.6.0" + "@parcel/types" "2.6.0" + "@parcel/utils" "2.6.0" + "@parcel/watcher" "^2.0.0" + "@parcel/workers" "2.6.0" + +"@parcel/graph@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/graph/-/graph-2.6.0.tgz#04f9660333e314a51af38483efefd766a5841bb0" + integrity sha512-rxrAzWm6rwbCRPbu0Z+zwMscpG8omffODniVWPlX2G0jgQGpjKsutBQ6RMfFIcfaQ4MzL3pIQOTf8bkjQOPsbg== + dependencies: + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/hash@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/hash/-/hash-2.6.0.tgz#c41364425e08d7e0ae5dae8b49ebfec2094124fe" + integrity sha512-YugWqhLxqK80Lo++3B3Kr5UPCHOdS8iI2zJ1jkzUeH9v6WUzbwWOnmPf6lN2S5m1BrIFFJd8Jc+CbEXWi8zoJA== + dependencies: + detect-libc "^1.0.3" + xxhash-wasm "^0.4.2" + +"@parcel/logger@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/logger/-/logger-2.6.0.tgz#f7aa26368e39573a5362997bb215f4a987c799e4" + integrity sha512-J1/7kPfSGBvMKSZdi0WCNuN0fIeiWxifnDGn7W/K8KhD422YwFJA8N046ps8nkDOPIXf1osnIECNp4GIR9oSYw== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/events" "2.6.0" + +"@parcel/markdown-ansi@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/markdown-ansi/-/markdown-ansi-2.6.0.tgz#69720735d27ca039e1e03f0277224ec5a99c0ef7" + integrity sha512-fyjkrJQQSfKTUFTTasdZ6WrAkDoQ2+DYDjj+3p+RncYyrIa9zArKx4IiRiipsvNdtMvP0/hTdK8F3BOJ3KSU/g== + dependencies: + chalk "^4.1.0" + +"@parcel/namer-default@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/namer-default/-/namer-default-2.6.0.tgz#1978397aabf13824f433157c683f64e1b6d37936" + integrity sha512-r8O12r7ozJBctnFxVdXbf/fK97GIdNj3hiiUNWlXEmED9sw6ZPcChaLcfot0/443g8i87JDmSTKJ8js2tuz5XA== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/plugin" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/node-resolver-core@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/node-resolver-core/-/node-resolver-core-2.6.0.tgz#2666897414274e0de72221f8ec34590f029ab95d" + integrity sha512-AJDj5DZbB58plv0li8bdVSD+zpnkHE36Om3TYyNn1jgXXwgBM64Er/9p8yQn356jBqTQMh7zlJqvbdIyOiMeMg== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/optimizer-terser@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/optimizer-terser/-/optimizer-terser-2.6.0.tgz#98179f7e5e4c74f80aaca7660d4a83d0ed03877e" + integrity sha512-oezRt6Lz/QqcVDXyMfFjzQc7n0ThJowLJ4Lyhu8rMh0ZJYzc4UCFCw/19d4nRnzE+Qg0vj3mQCpdkA9/64E44g== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/plugin" "2.6.0" + "@parcel/source-map" "^2.0.0" + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + terser "^5.2.0" + +"@parcel/package-manager@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/package-manager/-/package-manager-2.6.0.tgz#2d5dca646f2484ff6d643e1a2ed88cc48b25c6f6" + integrity sha512-AqFfdkbOw51q/3ia2mIsFTmrpYEyUb3k+2uYC5GsLMz3go6OGn7/Crz0lZLSclv5EtwpRg3TWr9yL7RekVN/Uw== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/fs" "2.6.0" + "@parcel/logger" "2.6.0" + "@parcel/types" "2.6.0" + "@parcel/utils" "2.6.0" + "@parcel/workers" "2.6.0" + semver "^5.7.1" + +"@parcel/packager-js@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/packager-js/-/packager-js-2.6.0.tgz#31810f10de497dd67e1912f83de0aba66db58173" + integrity sha512-Uz3pqIFchFfKszWnNGDgIwM1uwHHJp7Dts6VzS9lf/2RbRgZT0fmce+NPgnVO5MMKBHzdvm32ShT6gFAABF5Vw== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/hash" "2.6.0" + "@parcel/plugin" "2.6.0" + "@parcel/source-map" "^2.0.0" + "@parcel/utils" "2.6.0" + globals "^13.2.0" + nullthrows "^1.1.1" + +"@parcel/packager-raw@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/packager-raw/-/packager-raw-2.6.0.tgz#8ccf041acd102a38b2ffa02fd8ef634652255bd2" + integrity sha512-ktT6Qc/GgCq8H1+6y+AXufVzQj1s6KRoKf83qswCD0iY3MwCbJoEfc3IsB4K64FpHIL5Eu0z54IId+INvGbOYA== + dependencies: + "@parcel/plugin" "2.6.0" + +"@parcel/plugin@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/plugin/-/plugin-2.6.0.tgz#84fd9fffd7891027e4040be4b94647652fd46354" + integrity sha512-LzOaiK8R6eFEoov1cb3/W+o0XvXdI/VbDhMDl0L0II+/56M0UeayYtFP5QGTDn/fZqVlYfzPCtt3EMwdG7/dow== + dependencies: + "@parcel/types" "2.6.0" + +"@parcel/reporter-dev-server@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/reporter-dev-server/-/reporter-dev-server-2.6.0.tgz#8e692916c6684c3c04fecef058ddddae6b74121c" + integrity sha512-VvygsCA+uzWyijIV8zqU1gFyhAWknuaY4KIWhV4kCT8afRJwsLSwt/tpdaKDPuPU45h3tTsUdXH1wjaIk+dGeQ== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + +"@parcel/resolver-default@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/resolver-default/-/resolver-default-2.6.0.tgz#a80bc39c402abe0e78e3de8997ca2ea636c28a91" + integrity sha512-ATk9wXvy5GOHAqyHbnCnU11fUPTtf8dLjpgVqL5XylwugZnyBXbynoTWX4w8h6mffkVtdfmzTJx/o4Lresz9sA== + dependencies: + "@parcel/node-resolver-core" "2.6.0" + "@parcel/plugin" "2.6.0" + +"@parcel/runtime-browser-hmr@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/runtime-browser-hmr/-/runtime-browser-hmr-2.6.0.tgz#e817ead910f9ba572ed8f477447c862acbfe8d73" + integrity sha512-90xvv/10cFML5dAhClBEJZ/ExiBQVPqQsZcvRmVZmc5mpZVJMKattWCQrd7pAf7FDYl4JAcvsK3DTwvRT/oLNA== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + +"@parcel/runtime-js@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/runtime-js/-/runtime-js-2.6.0.tgz#a10c672c7f90360d5903180d0e2b808355708e80" + integrity sha512-R4tJAIT/SX7VBQ+f7WmeekREQzzLsmgP1j486uKhQNyYrpvsN0HnRbg5aqvZjEjkEmSeJR0mOlWtMK5/m+0yTA== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/runtime-react-refresh@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/runtime-react-refresh/-/runtime-react-refresh-2.6.0.tgz#52ae4d9acba3e1e3b20a4f2712ea140fac21aaaf" + integrity sha512-2sRd13gc2EbMV/O5n2NPVGGhKBasb1fDTXGEY8y7qi9xDKc+ewok/D83T+w243FhCPS9Pf3ur5GkbPlrJGcenQ== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + react-error-overlay "6.0.9" + react-refresh "^0.9.0" + +"@parcel/runtime-service-worker@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/runtime-service-worker/-/runtime-service-worker-2.6.0.tgz#10e90d02d83ebe763bb8de838a8f03eb3118aef9" + integrity sha512-nVlknGw5J5Bkd1Wr1TbyWHhUd9CmVVebaRg/lpfVKYhAuE/2r+3N0+J8qbEIgtTRcHaSV7wTNpg4weSWq46VeA== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + nullthrows "^1.1.1" + +"@parcel/source-map@^2.0.0": + version "2.0.5" + resolved "https://registry.npmmirror.com/@parcel/source-map/-/source-map-2.0.5.tgz#7dabcea0890914c390b8cf32e208d3f667e1ff38" + integrity sha512-DRVlCFKLpqBSIbMxUoVlHgfiv12HTW/U7nnhzw52YgzDVXUX9OA41dXS1PU0pJ1si+D1k8msATUC+AoldN43mg== + dependencies: + detect-libc "^1.0.3" + +"@parcel/transformer-js@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/transformer-js/-/transformer-js-2.6.0.tgz#b9f297e391a7091aaf0432135cd7f6c86e76301b" + integrity sha512-4v2r3EVdMKowBziVBW9HZqvAv88HaeiezkWyMX4wAfplo9jBtWEp99KEQINzSEdbXROR81M9oJjlGF5+yoVr/w== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/plugin" "2.6.0" + "@parcel/source-map" "^2.0.0" + "@parcel/utils" "2.6.0" + "@parcel/workers" "2.6.0" + "@swc/helpers" "^0.3.15" + browserslist "^4.6.6" + detect-libc "^1.0.3" + nullthrows "^1.1.1" + regenerator-runtime "^0.13.7" + semver "^5.7.1" + +"@parcel/transformer-json@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/transformer-json/-/transformer-json-2.6.0.tgz#c15fc774431bab4c7059b4013e0d1ca9b66fad5c" + integrity sha512-zb+TQAdHWdXijKcFhLe+5KN1O0IzXwW1gJhPr8DJEA3qhPaCsncsw5RCVjQlP3a7NXr1mMm1eMtO6bhIMqbXeA== + dependencies: + "@parcel/plugin" "2.6.0" + json5 "^2.2.0" + +"@parcel/transformer-raw@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/transformer-raw/-/transformer-raw-2.6.0.tgz#c0648e0f788bc71a26235788edc662f303f7c91b" + integrity sha512-QDirlWCS/qy0DQ3WvDIAnFP52n1TJW/uWH+4PGMNnX4/M3/2UchY8xp9CN0tx4NQ4g09S8o3gLlHvNxQqZxFrQ== + dependencies: + "@parcel/plugin" "2.6.0" + +"@parcel/transformer-react-refresh-wrap@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/transformer-react-refresh-wrap/-/transformer-react-refresh-wrap-2.6.0.tgz#8a3c2274549189c04440562ae4d3ca17ac4a861a" + integrity sha512-G34orfvLDUTumuerqNmA8T8NUHk+R0jwUjbVPO7gpB6VCVQ5ocTABdE9vN9Uu/cUsHij40TUFwqK4R9TFEBIEQ== + dependencies: + "@parcel/plugin" "2.6.0" + "@parcel/utils" "2.6.0" + react-refresh "^0.9.0" + +"@parcel/types@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/types/-/types-2.6.0.tgz#b9b7f93edaafcb77425e231a0b4662d3c8d61900" + integrity sha512-lAMYvOBfNEJMsPJ+plbB50305o0TwNrY1xX5RRIWBqwOa6bYmbW1ZljUk1tQvnkpIE4eAHQwnPR5Z2XWg18wGQ== + dependencies: + "@parcel/cache" "2.6.0" + "@parcel/diagnostic" "2.6.0" + "@parcel/fs" "2.6.0" + "@parcel/package-manager" "2.6.0" + "@parcel/source-map" "^2.0.0" + "@parcel/workers" "2.6.0" + utility-types "^3.10.0" + +"@parcel/utils@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/utils/-/utils-2.6.0.tgz#d2d42635ad5b398fa21b26940868e7ff30175c07" + integrity sha512-ElXz+QHtT1JQIucbQJBk7SzAGoOlBp4yodEQVvTKS7GA+hEGrSP/cmibl6qm29Rjtd0zgQsdd+2XmP3xvP2gQQ== + dependencies: + "@parcel/codeframe" "2.6.0" + "@parcel/diagnostic" "2.6.0" + "@parcel/hash" "2.6.0" + "@parcel/logger" "2.6.0" + "@parcel/markdown-ansi" "2.6.0" + "@parcel/source-map" "^2.0.0" + chalk "^4.1.0" + +"@parcel/watcher@^2.0.0": + version "2.0.5" + resolved "https://registry.npmmirror.com/@parcel/watcher/-/watcher-2.0.5.tgz#f913a54e1601b0aac972803829b0eece48de215b" + integrity sha512-x0hUbjv891omnkcHD7ZOhiyyUqUUR6MNjq89JhEI3BxppeKWAm6NPQsqqRrAkCJBogdT/o/My21sXtTI9rJIsw== + dependencies: + node-addon-api "^3.2.1" + node-gyp-build "^4.3.0" + +"@parcel/workers@2.6.0": + version "2.6.0" + resolved "https://registry.npmmirror.com/@parcel/workers/-/workers-2.6.0.tgz#09a53d62425d26eb1ee288371348c4dedf0347c9" + integrity sha512-3tcI2LF5fd/WZtSnSjyWdDE+G+FitdNrRgSObzSp+axHKMAM23sO0z7KY8s2SYCF40msdYbFUW8eI6JlYNJoWQ== + dependencies: + "@parcel/diagnostic" "2.6.0" + "@parcel/logger" "2.6.0" + "@parcel/types" "2.6.0" + "@parcel/utils" "2.6.0" + chrome-trace-event "^1.0.2" + nullthrows "^1.1.1" + +"@pmmmwh/react-refresh-webpack-plugin@^0.4.3": + version "0.4.3" + resolved "https://registry.npmmirror.com/@pmmmwh/react-refresh-webpack-plugin/-/react-refresh-webpack-plugin-0.4.3.tgz#1eec460596d200c0236bf195b078a5d1df89b766" + integrity sha512-br5Qwvh8D2OQqSXpd1g/xqXKnK0r+Jz6qVKBbWmpUcrbGOxUrf39V5oZ1876084CGn18uMdR5uvPqBv9UqtBjQ== + dependencies: + ansi-html "^0.0.7" + error-stack-parser "^2.0.6" + html-entities "^1.2.1" + native-url "^0.2.6" + schema-utils "^2.6.5" + source-map "^0.7.3" + +"@popperjs/core@^2.9.3": + version "2.11.5" + resolved "https://registry.npmmirror.com/@popperjs/core/-/core-2.11.5.tgz#db5a11bf66bdab39569719555b0f76e138d7bd64" + integrity sha512-9X2obfABZuDVLCgPK9aX0a/x4jaOEweTTWE2+9sr0Qqqevj2Uv5XorvusThmc9XGYpS9yI+fhh8RTafBtGposw== + +"@radix-ui/number@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/number/-/number-0.1.0.tgz#73ad13d5cc5f75fa5e147d72e5d5d5e50d688256" + integrity sha512-rpf6QiOWLHAkM4FEMYu9i+5Jr8cKT893+R4mPpcdsy4LD7omr9JfdOqj/h/xPA5+EcVrpMMlU6rrRYpUB5UI8g== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/primitive@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/primitive/-/primitive-0.1.0.tgz#6206b97d379994f0d1929809db035733b337e543" + integrity sha512-tqxZKybwN5Fa3VzZry4G6mXAAb9aAqKmPtnVbZpL0vsBwvOHTBwsjHVPXylocYLwEtBY9SCe665bYnNB515uoA== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-compose-refs@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/react-compose-refs/-/react-compose-refs-0.1.0.tgz#cff6e780a0f73778b976acff2c2a5b6551caab95" + integrity sha512-eyclbh+b77k+69Dk72q3694OHrn9B3QsoIRx7ywX341U9RK1ThgQjMFZoPtmZNQTksXHLNEiefR8hGVeFyInGg== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-context@0.1.1": + version "0.1.1" + resolved "https://registry.npmmirror.com/@radix-ui/react-context/-/react-context-0.1.1.tgz#06996829ea124d9a1bc1dbe3e51f33588fab0875" + integrity sha512-PkyVX1JsLBioeu0jB9WvRpDBBLtLZohVDT3BB5CTSJqActma8S8030P57mWZb4baZifMvN7KKWPAA40UmWKkQg== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-presence@0.1.2": + version "0.1.2" + resolved "https://registry.npmmirror.com/@radix-ui/react-presence/-/react-presence-0.1.2.tgz#9f11cce3df73cf65bc348e8b76d891f0d54c1fe3" + integrity sha512-3BRlFZraooIUfRlyN+b/Xs5hq1lanOOo/+3h6Pwu2GMFjkGKKa4Rd51fcqGqnVlbr3jYg+WLuGyAV4KlgqwrQw== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "0.1.0" + "@radix-ui/react-use-layout-effect" "0.1.0" + +"@radix-ui/react-primitive@0.1.4": + version "0.1.4" + resolved "https://registry.npmmirror.com/@radix-ui/react-primitive/-/react-primitive-0.1.4.tgz#6c233cf08b0cb87fecd107e9efecb3f21861edc1" + integrity sha512-6gSl2IidySupIMJFjYnDIkIWRyQdbu/AHK7rbICPani+LW4b0XdxBXc46og/iZvuwW8pjCS8I2SadIerv84xYA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-slot" "0.1.2" + +"@radix-ui/react-scroll-area@^0.1.1": + version "0.1.4" + resolved "https://registry.npmmirror.com/@radix-ui/react-scroll-area/-/react-scroll-area-0.1.4.tgz#be1d32c113ee9f64e3d2e7ee3983d98f00b42038" + integrity sha512-QHxRsjy+hsHwQYJ9cCNgSJ5+6ioZu1KhwD1UOXoHNciuFGMX08v+uJPKXIz+ySv03Rx6cOz6f/Fk5aPHRMFi/A== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/number" "0.1.0" + "@radix-ui/primitive" "0.1.0" + "@radix-ui/react-compose-refs" "0.1.0" + "@radix-ui/react-context" "0.1.1" + "@radix-ui/react-presence" "0.1.2" + "@radix-ui/react-primitive" "0.1.4" + "@radix-ui/react-use-callback-ref" "0.1.0" + "@radix-ui/react-use-direction" "0.1.0" + "@radix-ui/react-use-layout-effect" "0.1.0" + +"@radix-ui/react-slot@0.1.2": + version "0.1.2" + resolved "https://registry.npmmirror.com/@radix-ui/react-slot/-/react-slot-0.1.2.tgz#e6f7ad9caa8ce81cc8d532c854c56f9b8b6307c8" + integrity sha512-ADkqfL+agEzEguU3yS26jfB50hRrwf7U4VTwAOZEmi/g+ITcBWe12yM46ueS/UCIMI9Py+gFUaAdxgxafFvY2Q== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "0.1.0" + +"@radix-ui/react-use-callback-ref@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-0.1.0.tgz#934b6e123330f5b3a6b116460e6662cbc663493f" + integrity sha512-Va041McOFFl+aV+sejvl0BS2aeHx86ND9X/rVFmEFQKTXCp6xgUK0NGUAGcgBlIjnJSbMYPGEk1xKSSlVcN2Aw== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-use-direction@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/react-use-direction/-/react-use-direction-0.1.0.tgz#97ac1d52e497c974389e7988f809238ed72e7df7" + integrity sha512-NajpY/An9TCPSfOVkgWIdXJV+VuWl67PxB6kOKYmtNAFHvObzIoh8o0n9sAuwSAyFCZVq211FEf9gvVDRhOyiA== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-use-layout-effect@0.1.0": + version "0.1.0" + resolved "https://registry.npmmirror.com/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-0.1.0.tgz#ebf71bd6d2825de8f1fbb984abf2293823f0f223" + integrity sha512-+wdeS51Y+E1q1Wmd+1xSSbesZkpVj4jsg0BojCbopWvgq5iBvixw5vgemscdh58ep98BwUbsFYnrywFhV9yrVg== + dependencies: + "@babel/runtime" "^7.13.10" + +"@sideway/address@^4.1.3": + version "4.1.4" + resolved "https://registry.npmmirror.com/@sideway/address/-/address-4.1.4.tgz#03dccebc6ea47fdc226f7d3d1ad512955d4783f0" + integrity sha512-7vwq+rOHVWjyXxVlR76Agnvhy8I9rpzjosTESvmhNeXOXdZZB15Fl+TI9x1SiHZH5Jv2wTGduSxFDIaq0m3DUw== + dependencies: + "@hapi/hoek" "^9.0.0" + +"@sideway/formula@^3.0.0": + version "3.0.0" + resolved "https://registry.npmmirror.com/@sideway/formula/-/formula-3.0.0.tgz#fe158aee32e6bd5de85044be615bc08478a0a13c" + integrity sha512-vHe7wZ4NOXVfkoRb8T5otiENVlT7a3IAiw7H5M2+GO+9CDgcVUUsX1zalAztCmwyOr2RUTGJdgB+ZvSVqmdHmg== + +"@sideway/pinpoint@^2.0.0": + version "2.0.0" + resolved "https://registry.npmmirror.com/@sideway/pinpoint/-/pinpoint-2.0.0.tgz#cff8ffadc372ad29fd3f78277aeb29e632cc70df" + integrity sha512-RNiOoTPkptFtSVzQevY/yWtZwf/RxyVnPy/OcA9HBM3MlGDnBEYL5B41H0MTn0Uec8Hi+2qUtTfG2WWZBmMejQ== + +"@sindresorhus/is@^0.14.0": + version "0.14.0" + resolved "https://registry.npmmirror.com/@sindresorhus/is/-/is-0.14.0.tgz#9fb3a3cf3132328151f353de4632e01e52102bea" + integrity sha512-9NET910DNaIPngYnLLPeg+Ogzqsi9uM4mSboU5y6p8S5DzMTVEsJZrawi+BoDNUVBa2DhJqQYUFvMDfgU062LQ== + +"@sindresorhus/is@^4.0.0": + version "4.6.0" + resolved "https://registry.npmmirror.com/@sindresorhus/is/-/is-4.6.0.tgz#3c7c9c46e678feefe7a2e5bb609d3dbd665ffb3f" + integrity sha512-t09vSN3MdfsyCHoFcTRCH/iUtG7OJ0CsjzB8cjAmKc/va/kIgeDI/TxsigdncE/4be734m0cvIYwNaV4i2XqAw== + +"@sindresorhus/slugify@^1.1.2": + version "1.1.2" + resolved "https://registry.npmmirror.com/@sindresorhus/slugify/-/slugify-1.1.2.tgz#c2c0129298b8caace2d9156176fe244d0e83156c" + integrity sha512-V9nR/W0Xd9TSGXpZ4iFUcFGhuOJtZX82Fzxj1YISlbSgKvIiNa7eLEZrT0vAraPOt++KHauIVNYgGRgjc13dXA== + dependencies: + "@sindresorhus/transliterate" "^0.1.1" + escape-string-regexp "^4.0.0" + +"@sindresorhus/transliterate@^0.1.1": + version "0.1.2" + resolved "https://registry.npmmirror.com/@sindresorhus/transliterate/-/transliterate-0.1.2.tgz#ffce368271d153550e87de81486004f2637425af" + integrity sha512-5/kmIOY9FF32nicXH+5yLNTX4NJ4atl7jRgqAJuIn/iyDFXBktOKDxCvyGE/EzmF4ngSUvjXxQUQlQiZ5lfw+w== + dependencies: + escape-string-regexp "^2.0.0" + lodash.deburr "^4.1.0" + +"@swc/helpers@^0.3.15": + version "0.3.17" + resolved "https://registry.npmmirror.com/@swc/helpers/-/helpers-0.3.17.tgz#7c1b91f43c77e2bba99492162a498d465ef253d5" + integrity sha512-tb7Iu+oZ+zWJZ3HJqwx8oNwSDIU440hmVMDPhpACWQWnrZHK99Bxs70gT1L2dnr5Hg50ZRWEFkQCAnOVVV0z1Q== + dependencies: + tslib "^2.4.0" + +"@szmarczak/http-timer@^1.1.2": + version "1.1.2" + resolved "https://registry.npmmirror.com/@szmarczak/http-timer/-/http-timer-1.1.2.tgz#b1665e2c461a2cd92f4c1bbf50d5454de0d4b421" + integrity sha512-XIB2XbzHTN6ieIjfIMV9hlVcfPU26s2vafYWQcZHWXHOxiaRZYEDKEwdl129Zyg50+foYV2jCgtrqSA6qNuNSA== + dependencies: + defer-to-connect "^1.0.1" + +"@szmarczak/http-timer@^4.0.5": + version "4.0.6" + resolved "https://registry.npmmirror.com/@szmarczak/http-timer/-/http-timer-4.0.6.tgz#b4a914bb62e7c272d4e5989fe4440f812ab1d807" + integrity sha512-4BAffykYOgO+5nzBWYwE3W90sBgLJoUPRWWcL8wlyiM8IB8ipJz3UMJ9KXQd1RKQXpKp8Tutn80HZtWsu2u76w== + dependencies: + defer-to-connect "^2.0.0" + +"@tokenizer/token@^0.3.0": + version "0.3.0" + resolved "https://registry.npmmirror.com/@tokenizer/token/-/token-0.3.0.tgz#fe98a93fe789247e998c75e74e9c7c63217aa276" + integrity sha512-OvjF+z51L3ov0OyAU0duzsYuvO01PH7x4t6DJx+guahgTnBHkhJdG7soQeTSFLWN3efnHyibZ4Z8l2EuWwJN3A== + +"@trysound/sax@0.2.0": + version "0.2.0" + resolved "https://registry.npmmirror.com/@trysound/sax/-/sax-0.2.0.tgz#cccaab758af56761eb7bf37af6f03f326dd798ad" + integrity sha512-L7z9BgrNEcYyUYtF+HaEfiS5ebkh9jXqbszz7pC0hRBPaatV0XjSD3+eHrpqFemQfgwiFF0QPIarnIihIDn7OA== + +"@turist/fetch@^7.1.7": + version "7.2.0" + resolved "https://registry.npmmirror.com/@turist/fetch/-/fetch-7.2.0.tgz#57df869df1cd9b299588554eec4b8543effcc714" + integrity sha512-2x7EGw+6OJ29phunsbGvtxlNmSfcuPcyYudkMbi8gARCP9eJ1CtuMvnVUHL//O9Ixi9SJiug8wNt6lj86pN8XQ== + dependencies: + "@types/node-fetch" "2" + +"@turist/time@^0.0.2": + version "0.0.2" + resolved "https://registry.npmmirror.com/@turist/time/-/time-0.0.2.tgz#32fe0ce708ea0f4512776bd313409f1459976dda" + integrity sha512-qLOvfmlG2vCVw5fo/oz8WAZYlpe5a5OurgTj3diIxJCdjRHpapC+vQCz3er9LV79Vcat+DifBjeAhOAdmndtDQ== + +"@types/cacheable-request@^6.0.1": + version "6.0.2" + resolved "https://registry.npmmirror.com/@types/cacheable-request/-/cacheable-request-6.0.2.tgz#c324da0197de0a98a2312156536ae262429ff6b9" + integrity sha512-B3xVo+dlKM6nnKTcmm5ZtY/OL8bOAOd2Olee9M1zft65ox50OzjEHW91sDiU9j6cvW8Ejg1/Qkf4xd2kugApUA== + dependencies: + "@types/http-cache-semantics" "*" + "@types/keyv" "*" + "@types/node" "*" + "@types/responselike" "*" + +"@types/common-tags@^1.8.1": + version "1.8.1" + resolved "https://registry.npmmirror.com/@types/common-tags/-/common-tags-1.8.1.tgz#a5a49ca5ebbb58e0f8947f3ec98950c8970a68a9" + integrity sha512-20R/mDpKSPWdJs5TOpz3e7zqbeCNuMCPhV7Yndk9KU2Rbij2r5W4RzwDPkzC+2lzUqXYu9rFzTktCBnDjHuNQg== + +"@types/component-emitter@^1.2.10": + version "1.2.11" + resolved "https://registry.npmmirror.com/@types/component-emitter/-/component-emitter-1.2.11.tgz#50d47d42b347253817a39709fef03ce66a108506" + integrity sha512-SRXjM+tfsSlA9VuG8hGO2nft2p8zjXCK1VcC6N4NXbBbYbSia9kzCChYQajIjzIqOOOuh5Ock6MmV2oux4jDZQ== + +"@types/configstore@^2.1.1": + version "2.1.1" + resolved "https://registry.npmmirror.com/@types/configstore/-/configstore-2.1.1.tgz#cd1e8553633ad3185c3f2f239ecff5d2643e92b6" + integrity sha512-YY+hm3afkDHeSM2rsFXxeZtu0garnusBWNG1+7MknmDWQHqcH2w21/xOU9arJUi8ch4qyFklidANLCu3ihhVwQ== + +"@types/cookie@^0.4.0": + version "0.4.1" + resolved "https://registry.npmmirror.com/@types/cookie/-/cookie-0.4.1.tgz#bfd02c1f2224567676c1545199f87c3a861d878d" + integrity sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q== + +"@types/cors@^2.8.8": + version "2.8.12" + resolved "https://registry.npmmirror.com/@types/cors/-/cors-2.8.12.tgz#6b2c510a7ad7039e98e7b8d3d6598f4359e5c080" + integrity sha512-vt+kDhq/M2ayberEtJcIN/hxXy1Pk+59g2FV/ZQceeaTyCtCucjL2Q7FXlFjtWn4n15KCr1NE2lNNFhp0lEThw== + +"@types/debug@^0.0.30": + version "0.0.30" + resolved "https://registry.npmmirror.com/@types/debug/-/debug-0.0.30.tgz#dc1e40f7af3b9c815013a7860e6252f6352a84df" + integrity sha512-orGL5LXERPYsLov6CWs3Fh6203+dXzJkR7OnddIr2514Hsecwc8xRpzCapshBbKFImCsvS/mk6+FWiN5LyZJAQ== + +"@types/eslint-scope@^3.7.3": + version "3.7.3" + resolved "https://registry.npmmirror.com/@types/eslint-scope/-/eslint-scope-3.7.3.tgz#125b88504b61e3c8bc6f870882003253005c3224" + integrity sha512-PB3ldyrcnAicT35TWPs5IcwKD8S333HMaa2VVv4+wdvebJkjWuW/xESoB8IwRcog8HYVYamb1g/R31Qv5Bx03g== + dependencies: + "@types/eslint" "*" + "@types/estree" "*" + +"@types/eslint@*": + version "8.4.3" + resolved "https://registry.npmmirror.com/@types/eslint/-/eslint-8.4.3.tgz#5c92815a3838b1985c90034cd85f26f59d9d0ece" + integrity sha512-YP1S7YJRMPs+7KZKDb9G63n8YejIwW9BALq7a5j2+H4yl6iOv9CB29edho+cuFRrvmJbbaH2yiVChKLJVysDGw== + dependencies: + "@types/estree" "*" + "@types/json-schema" "*" + +"@types/eslint@^7.28.2": + version "7.29.0" + resolved "https://registry.npmmirror.com/@types/eslint/-/eslint-7.29.0.tgz#e56ddc8e542815272720bb0b4ccc2aff9c3e1c78" + integrity sha512-VNcvioYDH8/FxaeTKkM4/TiTwt6pBV9E3OfGmvaw8tPl0rrHCJ4Ll15HRT+pMiFAf/MLQvAzC+6RzUMEL9Ceng== + dependencies: + "@types/estree" "*" + "@types/json-schema" "*" + +"@types/estree@*", "@types/estree@^0.0.51": + version "0.0.51" + resolved "https://registry.npmmirror.com/@types/estree/-/estree-0.0.51.tgz#cfd70924a25a3fd32b218e5e420e6897e1ac4f40" + integrity sha512-CuPgU6f3eT/XgKKPqKd/gLZV1Xmvf1a2R5POBOGQa6uv82xpls89HU5zKeVoyR8XzHd1RGNOlQlvUe3CFkjWNQ== + +"@types/get-port@^3.2.0": + version "3.2.0" + resolved "https://registry.npmmirror.com/@types/get-port/-/get-port-3.2.0.tgz#f9e0a11443cc21336470185eae3dfba4495d29bc" + integrity sha512-TiNg8R1kjDde5Pub9F9vCwZA/BNW9HeXP5b9j7Qucqncy/McfPZ6xze/EyBdXS5FhMIGN6Fx3vg75l5KHy3V1Q== + +"@types/glob@*": + version "7.2.0" + resolved "https://registry.npmmirror.com/@types/glob/-/glob-7.2.0.tgz#bc1b5bf3aa92f25bd5dd39f35c57361bdce5b2eb" + integrity sha512-ZUxbzKl0IfJILTS6t7ip5fQQM/J3TJYubDm3nMbgubNNYS62eXeUpoLUC8/7fJNiFYHTrGPQn7hspDUzIHX3UA== + dependencies: + "@types/minimatch" "*" + "@types/node" "*" + +"@types/glob@^5.0.34": + version "5.0.37" + resolved "https://registry.npmmirror.com/@types/glob/-/glob-5.0.37.tgz#d0982abc88f9aebbd62099d3d70440cbcea692de" + integrity sha512-ATA/xrS7CZ3A2WCPVY4eKdNpybq56zqlTirnHhhyOztZM/lPxJzusOBI3BsaXbu6FrUluqzvMlI4sZ6BDYMlMg== + dependencies: + "@types/minimatch" "*" + "@types/node" "*" + +"@types/hast@^2.0.0": + version "2.3.4" + resolved "https://registry.npmmirror.com/@types/hast/-/hast-2.3.4.tgz#8aa5ef92c117d20d974a82bdfb6a648b08c0bafc" + integrity sha512-wLEm0QvaoawEDoTRwzTXp4b4jpwiJDvR5KMnFnVodm3scufTlBOWRD6N1OBf9TZMhjlNsSfcO5V+7AF4+Vy+9g== + dependencies: + "@types/unist" "*" + +"@types/http-cache-semantics@*": + version "4.0.1" + resolved "https://registry.npmmirror.com/@types/http-cache-semantics/-/http-cache-semantics-4.0.1.tgz#0ea7b61496902b95890dc4c3a116b60cb8dae812" + integrity sha512-SZs7ekbP8CN0txVG2xVRH6EgKmEm31BOxA07vkFaETzZz1xh+cbt8BcI0slpymvwhx5dlFnQG2rTlPVQn+iRPQ== + +"@types/http-proxy@^1.17.7": + version "1.17.9" + resolved "https://registry.npmmirror.com/@types/http-proxy/-/http-proxy-1.17.9.tgz#7f0e7931343761efde1e2bf48c40f02f3f75705a" + integrity sha512-QsbSjA/fSk7xB+UXlCT3wHBy5ai9wOcNDWwZAtud+jXhwOM3l+EYZh8Lng4+/6n8uar0J7xILzqftJdJ/Wdfkw== + dependencies: + "@types/node" "*" + +"@types/json-buffer@~3.0.0": + version "3.0.0" + resolved "https://registry.npmmirror.com/@types/json-buffer/-/json-buffer-3.0.0.tgz#85c1ff0f0948fc159810d4b5be35bf8c20875f64" + integrity sha512-3YP80IxxFJB4b5tYC2SUPwkg0XQLiu0nWvhRgEatgjf+29IcWO9X1k8xRv5DGssJ/lCrjYTjQPcobJr2yWIVuQ== + +"@types/json-schema@*", "@types/json-schema@^7.0.4", "@types/json-schema@^7.0.5", "@types/json-schema@^7.0.7", "@types/json-schema@^7.0.8": + version "7.0.11" + resolved "https://registry.npmmirror.com/@types/json-schema/-/json-schema-7.0.11.tgz#d421b6c527a3037f7c84433fd2c4229e016863d3" + integrity sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ== + +"@types/json5@^0.0.29": + version "0.0.29" + resolved "https://registry.npmmirror.com/@types/json5/-/json5-0.0.29.tgz#ee28707ae94e11d2b827bcbe5270bcea7f3e71ee" + integrity sha512-dRLjCWHYg4oaA77cxO64oO+7JwCwnIzkZPdrrC71jQmQtlhM556pwKo5bUzqvZndkVbeFLIIi+9TC40JNF5hNQ== + +"@types/keyv@*": + version "3.1.4" + resolved "https://registry.npmmirror.com/@types/keyv/-/keyv-3.1.4.tgz#3ccdb1c6751b0c7e52300bcdacd5bcbf8faa75b6" + integrity sha512-BQ5aZNSCpj7D6K2ksrRCTmKRLEpnPvWDiLPfoGyhZ++8YtiK9d/3DBKPJgry359X/P1PfruyYwvnvwFjuEiEIg== + dependencies: + "@types/node" "*" + +"@types/lodash@^4.14.92": + version "4.14.182" + resolved "https://registry.npmmirror.com/@types/lodash/-/lodash-4.14.182.tgz#05301a4d5e62963227eaafe0ce04dd77c54ea5c2" + integrity sha512-/THyiqyQAP9AfARo4pF+aCGcyiQ94tX/Is2I7HofNRqoYLgN1PBoOWu2/zTA5zMxzP5EFutMtWtGAFRKUe961Q== + +"@types/mdast@^3.0.0", "@types/mdast@^3.0.3": + version "3.0.10" + resolved "https://registry.npmmirror.com/@types/mdast/-/mdast-3.0.10.tgz#4724244a82a4598884cbbe9bcfd73dff927ee8af" + integrity sha512-W864tg/Osz1+9f4lrGTZpCSO5/z4608eUp19tbozkq2HJK6i3z1kT0H9tlADXuYIb1YYOBByU4Jsqkk75q48qA== + dependencies: + "@types/unist" "*" + +"@types/minimatch@*": + version "3.0.5" + resolved "https://registry.npmmirror.com/@types/minimatch/-/minimatch-3.0.5.tgz#1001cc5e6a3704b83c236027e77f2f58ea010f40" + integrity sha512-Klz949h02Gz2uZCMGwDUSDS1YBlTdDDgbWHi+81l29tQALUtvz4rAYi5uoVhE5Lagoq6DeqAUlbrHvW/mXDgdQ== + +"@types/minimist@^1.2.0": + version "1.2.2" + resolved "https://registry.npmmirror.com/@types/minimist/-/minimist-1.2.2.tgz#ee771e2ba4b3dc5b372935d549fd9617bf345b8c" + integrity sha512-jhuKLIRrhvCPLqwPcx6INqmKeiA5EWrsCOPhrlFSrbrmU4ZMPjj5Ul/oLCMDO98XRUIwVm78xICz4EPCektzeQ== + +"@types/mkdirp@^0.5.2": + version "0.5.2" + resolved "https://registry.npmmirror.com/@types/mkdirp/-/mkdirp-0.5.2.tgz#503aacfe5cc2703d5484326b1b27efa67a339c1f" + integrity sha512-U5icWpv7YnZYGsN4/cmh3WD2onMY0aJIiTE6+51TwJCttdHvtCYmkBNOobHlXwrJRL0nkH9jH4kD+1FAdMN4Tg== + dependencies: + "@types/node" "*" + +"@types/node-fetch@2": + version "2.6.1" + resolved "https://registry.npmmirror.com/@types/node-fetch/-/node-fetch-2.6.1.tgz#8f127c50481db65886800ef496f20bbf15518975" + integrity sha512-oMqjURCaxoSIsHSr1E47QHzbmzNR5rK8McHuNb11BOM9cHcIK3Avy0s/b2JlXHoQGTYS3NsvWzV1M0iK7l0wbA== + dependencies: + "@types/node" "*" + form-data "^3.0.0" + +"@types/node@*", "@types/node@>=10.0.0": + version "17.0.43" + resolved "https://registry.npmmirror.com/@types/node/-/node-17.0.43.tgz#7f16898cdd791c9d64069000ad448b47b3ca8353" + integrity sha512-jnUpgw8fL9kP2iszfIDyBQtw5Mf4/XSqy0Loc1J9pI14ejL83XcCEvSf50Gs/4ET0I9VCCDoOfufQysj0S66xA== + +"@types/node@16.9.1": + version "16.9.1" + resolved "https://registry.npmmirror.com/@types/node/-/node-16.9.1.tgz#0611b37db4246c937feef529ddcc018cf8e35708" + integrity sha512-QpLcX9ZSsq3YYUUnD3nFDY8H7wctAhQj/TFKL8Ya8v5fMm3CFXxo8zStsLAl780ltoYoo1WvKUVGBQK+1ifr7g== + +"@types/node@^8.5.7": + version "8.10.66" + resolved "https://registry.npmmirror.com/@types/node/-/node-8.10.66.tgz#dd035d409df322acc83dff62a602f12a5783bbb3" + integrity sha512-tktOkFUA4kXx2hhhrB8bIFb5TbwzS4uOhKEmwiD+NoiL0qtP2OQ9mFldbgD4dV1djrlBYP6eBuQZiWjuHUpqFw== + +"@types/normalize-package-data@^2.4.0": + version "2.4.1" + resolved "https://registry.npmmirror.com/@types/normalize-package-data/-/normalize-package-data-2.4.1.tgz#d3357479a0fdfdd5907fe67e17e0a85c906e1301" + integrity sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw== + +"@types/parse-json@^4.0.0": + version "4.0.0" + resolved "https://registry.npmmirror.com/@types/parse-json/-/parse-json-4.0.0.tgz#2f8bb441434d163b35fb8ffdccd7138927ffb8c0" + integrity sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA== + +"@types/parse5@^5.0.0": + version "5.0.3" + resolved "https://registry.npmmirror.com/@types/parse5/-/parse5-5.0.3.tgz#e7b5aebbac150f8b5fdd4a46e7f0bd8e65e19109" + integrity sha512-kUNnecmtkunAoQ3CnjmMkzNU/gtxG8guhi+Fk2U/kOpIKjIMKnXGp4IJCgQJrXSgMsWYimYG4TGjz/UzbGEBTw== + +"@types/prop-types@*": + version "15.7.5" + resolved "https://registry.npmmirror.com/@types/prop-types/-/prop-types-15.7.5.tgz#5f19d2b85a98e9558036f6a3cacc8819420f05cf" + integrity sha512-JCB8C6SnDoQf0cNycqd/35A7MjcnK+ZTqE7judS6o7utxUCg6imJg3QK2qzHKszlTjcj2cn+NwMB2i96ubpj7w== + +"@types/q@^1.5.1": + version "1.5.5" + resolved "https://registry.npmmirror.com/@types/q/-/q-1.5.5.tgz#75a2a8e7d8ab4b230414505d92335d1dcb53a6df" + integrity sha512-L28j2FcJfSZOnL1WBjDYp2vUHCeIFlyYI/53EwD/rKUBQ7MtUUfbQWiyKJGpcnv4/WgrhWsFKrcPstcAt/J0tQ== + +"@types/reach__router@^1.3.10": + version "1.3.10" + resolved "https://registry.npmmirror.com/@types/reach__router/-/reach__router-1.3.10.tgz#141d500213a452d9d9d71d5ad96c4104094f55a3" + integrity sha512-iHAFGaVOrWi00/q7oBybggGsz5TOmwOW4M1H9sT7i9lly4qFC8XOgsdf6jUsoaOz2sknFHALEtZqCoDbokdJ2Q== + dependencies: + "@types/react" "*" + +"@types/react@*": + version "18.0.12" + resolved "https://registry.npmmirror.com/@types/react/-/react-18.0.12.tgz#cdaa209d0a542b3fcf69cf31a03976ec4cdd8840" + integrity sha512-duF1OTASSBQtcigUvhuiTB1Ya3OvSy+xORCiEf20H0P0lzx+/KeVsA99U5UjLXSbyo1DRJDlLKqTeM1ngosqtg== + dependencies: + "@types/prop-types" "*" + "@types/scheduler" "*" + csstype "^3.0.2" + +"@types/responselike@*", "@types/responselike@^1.0.0": + version "1.0.0" + resolved "https://registry.npmmirror.com/@types/responselike/-/responselike-1.0.0.tgz#251f4fe7d154d2bad125abe1b429b23afd262e29" + integrity sha512-85Y2BjiufFzaMIlvJDvTTB8Fxl2xfLo4HgmHzVBz08w4wDePCTjYw66PdrolO0kzli3yam/YCgRufyo1DdQVTA== + dependencies: + "@types/node" "*" + +"@types/rimraf@^2.0.2": + version "2.0.5" + resolved "https://registry.npmmirror.com/@types/rimraf/-/rimraf-2.0.5.tgz#368fb04d59630b727fc05a74d2ca557f64a8ef98" + integrity sha512-YyP+VfeaqAyFmXoTh3HChxOQMyjByRMsHU7kc5KOJkSlXudhMhQIALbYV7rHh/l8d2lX3VUQzprrcAgWdRuU8g== + dependencies: + "@types/glob" "*" + "@types/node" "*" + +"@types/scheduler@*": + version "0.16.2" + resolved "https://registry.npmmirror.com/@types/scheduler/-/scheduler-0.16.2.tgz#1a62f89525723dde24ba1b01b092bf5df8ad4d39" + integrity sha512-hppQEBDmlwhFAXKJX2KnWLYu5yMfi91yazPb2l+lbJiwW+wdo1gNeRA+3RgNSO39WYX2euey41KEwnqesU2Jew== + +"@types/sharp@^0.30.0": + version "0.30.4" + resolved "https://registry.npmmirror.com/@types/sharp/-/sharp-0.30.4.tgz#7430b5fcf37f35dd860112c4cf6dcd6a1ba0011b" + integrity sha512-6oJEzKt7wZeS7e+6x9QFEOWGs0T/6of00+0onZGN1zSmcSjcTDZKgIGZ6YWJnHowpaKUCFBPH52mYljWqU32Eg== + dependencies: + "@types/node" "*" + +"@types/tmp@^0.0.33": + version "0.0.33" + resolved "https://registry.npmmirror.com/@types/tmp/-/tmp-0.0.33.tgz#1073c4bc824754ae3d10cfab88ab0237ba964e4d" + integrity sha512-gVC1InwyVrO326wbBZw+AO3u2vRXz/iRWq9jYhpG4W8LXyIgDv3ZmcLQ5Q4Gs+gFMyqx+viFoFT+l3p61QFCmQ== + +"@types/unist@*", "@types/unist@^2.0.0", "@types/unist@^2.0.2", "@types/unist@^2.0.3": + version "2.0.6" + resolved "https://registry.npmmirror.com/@types/unist/-/unist-2.0.6.tgz#250a7b16c3b91f672a24552ec64678eeb1d3a08d" + integrity sha512-PBjIUxZHOuj0R15/xuwJYjFi+KZdNFrehocChv4g5hu6aFroHue8m0lBP0POdK2nKzbw0cgV1mws8+V/JAcEkQ== + +"@types/websocket@1.0.2": + version "1.0.2" + resolved "https://registry.npmmirror.com/@types/websocket/-/websocket-1.0.2.tgz#d2855c6a312b7da73ed16ba6781815bf30c6187a" + integrity sha512-B5m9aq7cbbD/5/jThEr33nUY8WEfVi6A2YKCTOvw5Ldy7mtsOkqRvGjnzy6g7iMMDsgu7xREuCzqATLDLQVKcQ== + dependencies: + "@types/node" "*" + +"@types/yoga-layout@1.9.2": + version "1.9.2" + resolved "https://registry.npmmirror.com/@types/yoga-layout/-/yoga-layout-1.9.2.tgz#efaf9e991a7390dc081a0b679185979a83a9639a" + integrity sha512-S9q47ByT2pPvD65IvrWp7qppVMpk9WGMbVq9wbWZOHg6tnXSD4vyhao6nOSBwwfDdV2p3Kx9evA9vI+XWTfDvw== + +"@typescript-eslint/eslint-plugin@^4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/eslint-plugin/-/eslint-plugin-4.33.0.tgz#c24dc7c8069c7706bc40d99f6fa87edcb2005276" + integrity sha512-aINiAxGVdOl1eJyVjaWn/YcVAq4Gi/Yo35qHGCnqbWVz61g39D0h23veY/MA0rFFGfxK7TySg2uwDeNv+JgVpg== + dependencies: + "@typescript-eslint/experimental-utils" "4.33.0" + "@typescript-eslint/scope-manager" "4.33.0" + debug "^4.3.1" + functional-red-black-tree "^1.0.1" + ignore "^5.1.8" + regexpp "^3.1.0" + semver "^7.3.5" + tsutils "^3.21.0" + +"@typescript-eslint/experimental-utils@4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/experimental-utils/-/experimental-utils-4.33.0.tgz#6f2a786a4209fa2222989e9380b5331b2810f7fd" + integrity sha512-zeQjOoES5JFjTnAhI5QY7ZviczMzDptls15GFsI6jyUOq0kOf9+WonkhtlIhh0RgHRnqj5gdNxW5j1EvAyYg6Q== + dependencies: + "@types/json-schema" "^7.0.7" + "@typescript-eslint/scope-manager" "4.33.0" + "@typescript-eslint/types" "4.33.0" + "@typescript-eslint/typescript-estree" "4.33.0" + eslint-scope "^5.1.1" + eslint-utils "^3.0.0" + +"@typescript-eslint/parser@^4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/parser/-/parser-4.33.0.tgz#dfe797570d9694e560528d18eecad86c8c744899" + integrity sha512-ZohdsbXadjGBSK0/r+d87X0SBmKzOq4/S5nzK6SBgJspFo9/CUDJ7hjayuze+JK7CZQLDMroqytp7pOcFKTxZA== + dependencies: + "@typescript-eslint/scope-manager" "4.33.0" + "@typescript-eslint/types" "4.33.0" + "@typescript-eslint/typescript-estree" "4.33.0" + debug "^4.3.1" + +"@typescript-eslint/scope-manager@4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/scope-manager/-/scope-manager-4.33.0.tgz#d38e49280d983e8772e29121cf8c6e9221f280a3" + integrity sha512-5IfJHpgTsTZuONKbODctL4kKuQje/bzBRkwHE8UOZ4f89Zeddg+EGZs8PD8NcN4LdM3ygHWYB3ukPAYjvl/qbQ== + dependencies: + "@typescript-eslint/types" "4.33.0" + "@typescript-eslint/visitor-keys" "4.33.0" + +"@typescript-eslint/types@4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/types/-/types-4.33.0.tgz#a1e59036a3b53ae8430ceebf2a919dc7f9af6d72" + integrity sha512-zKp7CjQzLQImXEpLt2BUw1tvOMPfNoTAfb8l51evhYbOEEzdWyQNmHWWGPR6hwKJDAi+1VXSBmnhL9kyVTTOuQ== + +"@typescript-eslint/typescript-estree@4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/typescript-estree/-/typescript-estree-4.33.0.tgz#0dfb51c2908f68c5c08d82aefeaf166a17c24609" + integrity sha512-rkWRY1MPFzjwnEVHsxGemDzqqddw2QbTJlICPD9p9I9LfsO8fdmfQPOX3uKfUaGRDFJbfrtm/sXhVXN4E+bzCA== + dependencies: + "@typescript-eslint/types" "4.33.0" + "@typescript-eslint/visitor-keys" "4.33.0" + debug "^4.3.1" + globby "^11.0.3" + is-glob "^4.0.1" + semver "^7.3.5" + tsutils "^3.21.0" + +"@typescript-eslint/visitor-keys@4.33.0": + version "4.33.0" + resolved "https://registry.npmmirror.com/@typescript-eslint/visitor-keys/-/visitor-keys-4.33.0.tgz#2a22f77a41604289b7a186586e9ec48ca92ef1dd" + integrity sha512-uqi/2aSz9g2ftcHWf8uLPJA70rUv6yuMW5Bohw+bwcuzaxQIHaKFZCKGoGXIrc9vkTJ3+0txM73K0Hq3d5wgIg== + dependencies: + "@typescript-eslint/types" "4.33.0" + eslint-visitor-keys "^2.0.0" + +"@vercel/webpack-asset-relocator-loader@^1.7.0": + version "1.7.2" + resolved "https://registry.npmmirror.com/@vercel/webpack-asset-relocator-loader/-/webpack-asset-relocator-loader-1.7.2.tgz#0210abd8d53b2799d53156dd0c18a4ef4e3b51cb" + integrity sha512-pdMwUawmAtH/LScbjKJq/y2+gZFggFMc2tlJrlPSrgKajvYPEis3L9QKcMyC9RN1Xos4ezAP5AJfRCNN6RMKCQ== + +"@webassemblyjs/ast@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/ast/-/ast-1.11.1.tgz#2bfd767eae1a6996f432ff7e8d7fc75679c0b6a7" + integrity sha512-ukBh14qFLjxTQNTXocdyksN5QdM28S1CxHt2rdskFyL+xFV7VremuBLVbmCePj+URalXBENx/9Lm7lnhihtCSw== + dependencies: + "@webassemblyjs/helper-numbers" "1.11.1" + "@webassemblyjs/helper-wasm-bytecode" "1.11.1" + +"@webassemblyjs/floating-point-hex-parser@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.1.tgz#f6c61a705f0fd7a6aecaa4e8198f23d9dc179e4f" + integrity sha512-iGRfyc5Bq+NnNuX8b5hwBrRjzf0ocrJPI6GWFodBFzmFnyvrQ83SHKhmilCU/8Jv67i4GJZBMhEzltxzcNagtQ== + +"@webassemblyjs/helper-api-error@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.1.tgz#1a63192d8788e5c012800ba6a7a46c705288fd16" + integrity sha512-RlhS8CBCXfRUR/cwo2ho9bkheSXG0+NwooXcc3PAILALf2QLdFyj7KGsKRbVc95hZnhnERon4kW/D3SZpp6Tcg== + +"@webassemblyjs/helper-buffer@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.1.tgz#832a900eb444884cde9a7cad467f81500f5e5ab5" + integrity sha512-gwikF65aDNeeXa8JxXa2BAk+REjSyhrNC9ZwdT0f8jc4dQQeDQ7G4m0f2QCLPJiMTTO6wfDmRmj/pW0PsUvIcA== + +"@webassemblyjs/helper-numbers@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.1.tgz#64d81da219fbbba1e3bd1bfc74f6e8c4e10a62ae" + integrity sha512-vDkbxiB8zfnPdNK9Rajcey5C0w+QJugEglN0of+kmO8l7lDb77AnlKYQF7aarZuCrv+l0UvqL+68gSDr3k9LPQ== + dependencies: + "@webassemblyjs/floating-point-hex-parser" "1.11.1" + "@webassemblyjs/helper-api-error" "1.11.1" + "@xtuc/long" "4.2.2" + +"@webassemblyjs/helper-wasm-bytecode@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.1.tgz#f328241e41e7b199d0b20c18e88429c4433295e1" + integrity sha512-PvpoOGiJwXeTrSf/qfudJhwlvDQxFgelbMqtq52WWiXC6Xgg1IREdngmPN3bs4RoO83PnL/nFrxucXj1+BX62Q== + +"@webassemblyjs/helper-wasm-section@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.1.tgz#21ee065a7b635f319e738f0dd73bfbda281c097a" + integrity sha512-10P9No29rYX1j7F3EVPX3JvGPQPae+AomuSTPiF9eBQeChHI6iqjMIwR9JmOJXwpnn/oVGDk7I5IlskuMwU/pg== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/helper-buffer" "1.11.1" + "@webassemblyjs/helper-wasm-bytecode" "1.11.1" + "@webassemblyjs/wasm-gen" "1.11.1" + +"@webassemblyjs/ieee754@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/ieee754/-/ieee754-1.11.1.tgz#963929e9bbd05709e7e12243a099180812992614" + integrity sha512-hJ87QIPtAMKbFq6CGTkZYJivEwZDbQUgYd3qKSadTNOhVY7p+gfP6Sr0lLRVTaG1JjFj+r3YchoqRYxNH3M0GQ== + dependencies: + "@xtuc/ieee754" "^1.2.0" + +"@webassemblyjs/leb128@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/leb128/-/leb128-1.11.1.tgz#ce814b45574e93d76bae1fb2644ab9cdd9527aa5" + integrity sha512-BJ2P0hNZ0u+Th1YZXJpzW6miwqQUGcIHT1G/sf72gLVD9DZ5AdYTqPNbHZh6K1M5VmKvFXwGSWZADz+qBWxeRw== + dependencies: + "@xtuc/long" "4.2.2" + +"@webassemblyjs/utf8@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/utf8/-/utf8-1.11.1.tgz#d1f8b764369e7c6e6bae350e854dec9a59f0a3ff" + integrity sha512-9kqcxAEdMhiwQkHpkNiorZzqpGrodQQ2IGrHHxCy+Ozng0ofyMA0lTqiLkVs1uzTRejX+/O0EOT7KxqVPuXosQ== + +"@webassemblyjs/wasm-edit@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.1.tgz#ad206ebf4bf95a058ce9880a8c092c5dec8193d6" + integrity sha512-g+RsupUC1aTHfR8CDgnsVRVZFJqdkFHpsHMfJuWQzWU3tvnLC07UqHICfP+4XyL2tnr1amvl1Sdp06TnYCmVkA== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/helper-buffer" "1.11.1" + "@webassemblyjs/helper-wasm-bytecode" "1.11.1" + "@webassemblyjs/helper-wasm-section" "1.11.1" + "@webassemblyjs/wasm-gen" "1.11.1" + "@webassemblyjs/wasm-opt" "1.11.1" + "@webassemblyjs/wasm-parser" "1.11.1" + "@webassemblyjs/wast-printer" "1.11.1" + +"@webassemblyjs/wasm-gen@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.1.tgz#86c5ea304849759b7d88c47a32f4f039ae3c8f76" + integrity sha512-F7QqKXwwNlMmsulj6+O7r4mmtAlCWfO/0HdgOxSklZfQcDu0TpLiD1mRt/zF25Bk59FIjEuGAIyn5ei4yMfLhA== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/helper-wasm-bytecode" "1.11.1" + "@webassemblyjs/ieee754" "1.11.1" + "@webassemblyjs/leb128" "1.11.1" + "@webassemblyjs/utf8" "1.11.1" + +"@webassemblyjs/wasm-opt@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.1.tgz#657b4c2202f4cf3b345f8a4c6461c8c2418985f2" + integrity sha512-VqnkNqnZlU5EB64pp1l7hdm3hmQw7Vgqa0KF/KCNO9sIpI6Fk6brDEiX+iCOYrvMuBWDws0NkTOxYEb85XQHHw== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/helper-buffer" "1.11.1" + "@webassemblyjs/wasm-gen" "1.11.1" + "@webassemblyjs/wasm-parser" "1.11.1" + +"@webassemblyjs/wasm-parser@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.1.tgz#86ca734534f417e9bd3c67c7a1c75d8be41fb199" + integrity sha512-rrBujw+dJu32gYB7/Lup6UhdkPx9S9SnobZzRVL7VcBH9Bt9bCBLEuX/YXOOtBsOZ4NQrRykKhffRWHvigQvOA== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/helper-api-error" "1.11.1" + "@webassemblyjs/helper-wasm-bytecode" "1.11.1" + "@webassemblyjs/ieee754" "1.11.1" + "@webassemblyjs/leb128" "1.11.1" + "@webassemblyjs/utf8" "1.11.1" + +"@webassemblyjs/wast-printer@1.11.1": + version "1.11.1" + resolved "https://registry.npmmirror.com/@webassemblyjs/wast-printer/-/wast-printer-1.11.1.tgz#d0c73beda8eec5426f10ae8ef55cee5e7084c2f0" + integrity sha512-IQboUWM4eKzWW+N/jij2sRatKMh99QEelo3Eb2q0qXkvPRISAj8Qxtmw5itwqK+TTkBuUIE45AxYPToqPtL5gg== + dependencies: + "@webassemblyjs/ast" "1.11.1" + "@xtuc/long" "4.2.2" + +"@xtuc/ieee754@^1.2.0": + version "1.2.0" + resolved "https://registry.npmmirror.com/@xtuc/ieee754/-/ieee754-1.2.0.tgz#eef014a3145ae477a1cbc00cd1e552336dceb790" + integrity sha512-DX8nKgqcGwsc0eJSqYt5lwP4DH5FlHnmuWWBRy7X0NcaGR0ZtuyeESgMwTYVEtxmsNGY+qit4QYT/MIYTOTPeA== + +"@xtuc/long@4.2.2": + version "4.2.2" + resolved "https://registry.npmmirror.com/@xtuc/long/-/long-4.2.2.tgz#d291c6a4e97989b5c61d9acf396ae4fe133a718d" + integrity sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ== + +abbrev@1: + version "1.1.1" + resolved "https://registry.npmmirror.com/abbrev/-/abbrev-1.1.1.tgz#f8f2c887ad10bf67f634f005b6987fed3179aac8" + integrity sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q== + +abort-controller@3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/abort-controller/-/abort-controller-3.0.0.tgz#eaf54d53b62bae4138e809ca225c8439a6efb392" + integrity sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg== + dependencies: + event-target-shim "^5.0.0" + +abortcontroller-polyfill@^1.1.9: + version "1.7.3" + resolved "https://registry.npmmirror.com/abortcontroller-polyfill/-/abortcontroller-polyfill-1.7.3.tgz#1b5b487bd6436b5b764fd52a612509702c3144b5" + integrity sha512-zetDJxd89y3X99Kvo4qFx8GKlt6GsvN3UcRZHwU6iFA/0KiOmhkTVhe8oRoTBiTVPZu09x3vCra47+w8Yz1+2Q== + +accepts@^1.3.7, accepts@~1.3.4, accepts@~1.3.5, accepts@~1.3.8: + version "1.3.8" + resolved "https://registry.npmmirror.com/accepts/-/accepts-1.3.8.tgz#0bf0be125b67014adcb0b0921e62db7bffe16b2e" + integrity sha512-PYAthTa2m2VKxuvSD3DPC/Gy+U+sOA1LAuT8mkmRuvw+NACSaeXEQ+NHcVF7rONl6qcaxV3Uuemwawk+7+SJLw== + dependencies: + mime-types "~2.1.34" + negotiator "0.6.3" + +acorn-import-assertions@^1.7.6: + version "1.8.0" + resolved "https://registry.npmmirror.com/acorn-import-assertions/-/acorn-import-assertions-1.8.0.tgz#ba2b5939ce62c238db6d93d81c9b111b29b855e9" + integrity sha512-m7VZ3jwz4eK6A4Vtt8Ew1/mNbP24u0FhdyfA7fSvnJR6LMdfOYnmuIrrJAgrYfYJ10F/otaHTtrtrtmHdMNzEw== + +acorn-jsx@^5.3.1: + version "5.3.2" + resolved "https://registry.npmmirror.com/acorn-jsx/-/acorn-jsx-5.3.2.tgz#7ed5bb55908b3b2f1bc55c6af1653bada7f07937" + integrity sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ== + +acorn-node@^1.8.2: + version "1.8.2" + resolved "https://registry.npmmirror.com/acorn-node/-/acorn-node-1.8.2.tgz#114c95d64539e53dede23de8b9d96df7c7ae2af8" + integrity sha512-8mt+fslDufLYntIoPAaIMUe/lrbrehIiwmR3t2k9LljIzoigEPF27eLk2hy8zSGzmR/ogr7zbRKINMo1u0yh5A== + dependencies: + acorn "^7.0.0" + acorn-walk "^7.0.0" + xtend "^4.0.2" + +acorn-walk@^7.0.0: + version "7.2.0" + resolved "https://registry.npmmirror.com/acorn-walk/-/acorn-walk-7.2.0.tgz#0de889a601203909b0fbe07b8938dc21d2e967bc" + integrity sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA== + +acorn@^7.0.0, acorn@^7.4.0: + version "7.4.1" + resolved "https://registry.npmmirror.com/acorn/-/acorn-7.4.1.tgz#feaed255973d2e77555b83dbc08851a6c63520fa" + integrity sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A== + +acorn@^8.4.1, acorn@^8.5.0: + version "8.7.1" + resolved "https://registry.npmmirror.com/acorn/-/acorn-8.7.1.tgz#0197122c843d1bf6d0a5e83220a788f278f63c30" + integrity sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A== + +address@1.1.2: + version "1.1.2" + resolved "https://registry.npmmirror.com/address/-/address-1.1.2.tgz#bf1116c9c758c51b7a933d296b72c221ed9428b6" + integrity sha512-aT6camzM4xEA54YVJYSqxz1kv4IHnQZRtThJJHhUMRExaU5spC7jX5ugSwTaTgJliIgs4VhZOk7htClvQ/LmRA== + +address@^1.0.1, address@^1.1.2: + version "1.2.0" + resolved "https://registry.npmmirror.com/address/-/address-1.2.0.tgz#d352a62c92fee90f89a693eccd2a8b2139ab02d9" + integrity sha512-tNEZYz5G/zYunxFm7sfhAxkXEuLj3K6BKwv6ZURlsF6yiUQ65z0Q2wZW9L5cPUl9ocofGvXOdFYbFHp0+6MOig== + +adjust-sourcemap-loader@3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/adjust-sourcemap-loader/-/adjust-sourcemap-loader-3.0.0.tgz#5ae12fb5b7b1c585e80bbb5a63ec163a1a45e61e" + integrity sha512-YBrGyT2/uVQ/c6Rr+t6ZJXniY03YtHGMJQYal368burRGYKqhx9qGTWqcBU5s1CwYY9E/ri63RYyG1IacMZtqw== + dependencies: + loader-utils "^2.0.0" + regex-parser "^2.2.11" + +ajv-keywords@^3.4.1, ajv-keywords@^3.5.2: + version "3.5.2" + resolved "https://registry.npmmirror.com/ajv-keywords/-/ajv-keywords-3.5.2.tgz#31f29da5ab6e00d1c2d329acf7b5929614d5014d" + integrity sha512-5p6WTN0DdTGVQk6VjcEju19IgaHudalcfabD7yhDGeA6bcQnmL+CpveLJq/3hvfwd1aof6L386Ougkx6RfyMIQ== + +ajv@^6.10.0, ajv@^6.12.2, ajv@^6.12.3, ajv@^6.12.4, ajv@^6.12.5: + version "6.12.6" + resolved "https://registry.npmmirror.com/ajv/-/ajv-6.12.6.tgz#baf5a62e802b07d977034586f8c3baf5adf26df4" + integrity sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g== + dependencies: + fast-deep-equal "^3.1.1" + fast-json-stable-stringify "^2.0.0" + json-schema-traverse "^0.4.1" + uri-js "^4.2.2" + +ajv@^8.0.1: + version "8.11.0" + resolved "https://registry.npmmirror.com/ajv/-/ajv-8.11.0.tgz#977e91dd96ca669f54a11e23e378e33b884a565f" + integrity sha512-wGgprdCvMalC0BztXvitD2hC04YffAvtsUn93JbGXYLAtCUO4xd17mCCZQxUOItiBwZvJScWo8NIvQMQ71rdpg== + dependencies: + fast-deep-equal "^3.1.1" + json-schema-traverse "^1.0.0" + require-from-string "^2.0.2" + uri-js "^4.2.2" + +amdefine@>=0.0.4: + version "1.0.1" + resolved "https://registry.npmmirror.com/amdefine/-/amdefine-1.0.1.tgz#4a5282ac164729e93619bcfd3ad151f817ce91f5" + integrity sha512-S2Hw0TtNkMJhIabBwIojKL9YHO5T0n5eNqWJ7Lrlel/zDbftQpxpapi8tZs3X1HWa+u+QeydGmzzNU0m09+Rcg== + +anser@^2.1.0: + version "2.1.1" + resolved "https://registry.npmmirror.com/anser/-/anser-2.1.1.tgz#8afae28d345424c82de89cc0e4d1348eb0c5af7c" + integrity sha512-nqLm4HxOTpeLOxcmB3QWmV5TcDFhW9y/fyQ+hivtDFcK4OQ+pQ5fzPnXHM1Mfcm0VkLtvVi1TCPr++Qy0Q/3EQ== + +ansi-align@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/ansi-align/-/ansi-align-3.0.1.tgz#0cdf12e111ace773a86e9a1fad1225c43cb19a59" + integrity sha512-IOfwwBF5iczOjp/WeY4YxyjqAFMQoZufdQWDd19SEExbVLNXqvpzSJ/M7Za4/sCPmQ0+GRquoA7bGcINcxew6w== + dependencies: + string-width "^4.1.0" + +ansi-colors@^4.1.1: + version "4.1.3" + resolved "https://registry.npmmirror.com/ansi-colors/-/ansi-colors-4.1.3.tgz#37611340eb2243e70cc604cad35d63270d48781b" + integrity sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw== + +ansi-escapes@^3.1.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/ansi-escapes/-/ansi-escapes-3.2.0.tgz#8780b98ff9dbf5638152d1f1fe5c1d7b4442976b" + integrity sha512-cBhpre4ma+U0T1oM5fXg7Dy1Jw7zzwv7lt/GoCpr+hDQJoYnKVPLL4dCvSEFMmQurOQvSrwT7SL/DAlhBI97RQ== + +ansi-escapes@^4.2.1: + version "4.3.2" + resolved "https://registry.npmmirror.com/ansi-escapes/-/ansi-escapes-4.3.2.tgz#6b2291d1db7d98b6521d5f1efa42d0f3a9feb65e" + integrity sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ== + dependencies: + type-fest "^0.21.3" + +ansi-html-community@0.0.8: + version "0.0.8" + resolved "https://registry.npmmirror.com/ansi-html-community/-/ansi-html-community-0.0.8.tgz#69fbc4d6ccbe383f9736934ae34c3f8290f1bf41" + integrity sha512-1APHAyr3+PCamwNw3bXCPp4HFLONZt/yIH0sZp0/469KWNTEy+qN5jQ3GVX6DMZ1UXAi34yVwtTeaG/HpBuuzw== + +ansi-html@^0.0.7: + version "0.0.7" + resolved "https://registry.npmmirror.com/ansi-html/-/ansi-html-0.0.7.tgz#813584021962a9e9e6fd039f940d12f56ca7859e" + integrity sha512-JoAxEa1DfP9m2xfB/y2r/aKcwXNlltr4+0QSBC4TrLfcxyvepX2Pv0t/xpgGV5bGsDzCYV8SzjWgyCW0T9yYbA== + +ansi-regex@^2.0.0: + version "2.1.1" + resolved "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-2.1.1.tgz#c3b33ab5ee360d86e0e628f0468ae7ef27d654df" + integrity sha512-TIGnTpdo+E3+pCyAluZvtED5p5wCqLdezCyhPZzKPcxvFplEt4i+W7OONCKgeZFT3+y5NZZfOOS/Bdcanm1MYA== + +ansi-regex@^4.1.0: + version "4.1.1" + resolved "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-4.1.1.tgz#164daac87ab2d6f6db3a29875e2d1766582dabed" + integrity sha512-ILlv4k/3f6vfQ4OoP2AGvirOktlQ98ZEL1k9FaQjxa3L1abBgbuTDAdPOpvbGncC0BTVQrl+OM8xZGK6tWXt7g== + +ansi-regex@^5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" + integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== + +ansi-styles@^2.2.1: + version "2.2.1" + resolved "https://registry.npmmirror.com/ansi-styles/-/ansi-styles-2.2.1.tgz#b432dd3358b634cf75e1e4664368240533c1ddbe" + integrity sha512-kmCevFghRiWM7HB5zTPULl4r9bVFSWjz62MhqizDGUrq2NWuNMQyuv4tHHoKJHs69M/MF64lEcHdYIocrdWQYA== + +ansi-styles@^3.2.0, ansi-styles@^3.2.1: + version "3.2.1" + resolved "https://registry.npmmirror.com/ansi-styles/-/ansi-styles-3.2.1.tgz#41fbb20243e50b12be0f04b8dedbf07520ce841d" + integrity sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA== + dependencies: + color-convert "^1.9.0" + +ansi-styles@^4.0.0, ansi-styles@^4.1.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/ansi-styles/-/ansi-styles-4.3.0.tgz#edd803628ae71c04c85ae7a0906edad34b648937" + integrity sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg== + dependencies: + color-convert "^2.0.1" + +antd@^4.21.0: + version "4.21.2" + resolved "https://registry.npmmirror.com/antd/-/antd-4.21.2.tgz#07cf60b3100cca0484ed2eee3f7068785d670a20" + integrity sha512-DmxCMoA0IQKdeLVdHZjrHlsiXGDGHBvN62Y/+S0PfMrcQo7HW4Ps7RxVayobsCMovbrhX3Fx+u1/ES292me5WQ== + dependencies: + "@ant-design/colors" "^6.0.0" + "@ant-design/icons" "^4.7.0" + "@ant-design/react-slick" "~0.29.1" + "@babel/runtime" "^7.12.5" + "@ctrl/tinycolor" "^3.4.0" + classnames "^2.2.6" + copy-to-clipboard "^3.2.0" + lodash "^4.17.21" + memoize-one "^6.0.0" + moment "^2.29.2" + rc-cascader "~3.6.0" + rc-checkbox "~2.3.0" + rc-collapse "~3.3.0" + rc-dialog "~8.9.0" + rc-drawer "~4.4.2" + rc-dropdown "~4.0.0" + rc-field-form "~1.26.1" + rc-image "~5.7.0" + rc-input "~0.0.1-alpha.5" + rc-input-number "~7.3.0" + rc-mentions "~1.8.0" + rc-menu "~9.6.0" + rc-motion "^2.5.1" + rc-notification "~4.6.0" + rc-pagination "~3.1.16" + rc-picker "~2.6.8" + rc-progress "~3.3.2" + rc-rate "~2.9.0" + rc-resize-observer "^1.2.0" + rc-segmented "~2.1.0" + rc-select "~14.1.1" + rc-slider "~10.0.0" + rc-steps "~4.1.0" + rc-switch "~3.2.0" + rc-table "~7.24.0" + rc-tabs "~11.16.0" + rc-textarea "~0.3.0" + rc-tooltip "~5.1.1" + rc-tree "~5.6.5" + rc-tree-select "~5.4.0" + rc-trigger "^5.2.10" + rc-upload "~4.3.0" + rc-util "^5.20.0" + scroll-into-view-if-needed "^2.2.25" + +any-base@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/any-base/-/any-base-1.1.0.tgz#ae101a62bc08a597b4c9ab5b7089d456630549fe" + integrity sha512-uMgjozySS8adZZYePpaWs8cxB9/kdzmpX6SgJZ+wbz1K5eYk5QMYDVJaZKhxyIHUdnnJkfR7SVgStgH7LkGUyg== + +anymatch@~3.1.2: + version "3.1.2" + resolved "https://registry.npmmirror.com/anymatch/-/anymatch-3.1.2.tgz#c0557c096af32f106198f4f4e2a383537e378716" + integrity sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg== + dependencies: + normalize-path "^3.0.0" + picomatch "^2.0.4" + +aos@^2.3.4: + version "2.3.4" + resolved "https://registry.npmmirror.com/aos/-/aos-2.3.4.tgz#eb1c29f5c1806a197cf6323080747547edd3db8e" + integrity sha512-zh/ahtR2yME4I51z8IttIt4lC1Nw0ktsFtmeDzID1m9naJnWXhCoARaCgNOGXb5CLy3zm+wqmRAEgMYB5E2HUw== + dependencies: + classlist-polyfill "^1.0.3" + lodash.debounce "^4.0.6" + lodash.throttle "^4.0.1" + +append-field@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/append-field/-/append-field-1.0.0.tgz#1e3440e915f0b1203d23748e78edd7b9b5b43e56" + integrity sha512-klpgFSWLW1ZEs8svjfb7g4qWY0YS5imI82dTg+QahUvJ8YqAY0P10Uk8tTyh9ZGuYEZEMaeJYCF5BFuX552hsw== + +application-config-path@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/application-config-path/-/application-config-path-0.1.0.tgz#193c5f0a86541a4c66fba1e2dc38583362ea5e8f" + integrity sha512-lljTpVvFteShrHuKRvweZfa9o/Nc34Y8r5/1Lqh/yyKaspRT2J3fkEiSSk1YLG8ZSVyU7yHysRy9zcDDS2aH1Q== + +aproba@^1.0.3: + version "1.2.0" + resolved "https://registry.npmmirror.com/aproba/-/aproba-1.2.0.tgz#6802e6264efd18c790a1b0d517f0f2627bf2c94a" + integrity sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw== + +arch@^2.1.1: + version "2.2.0" + resolved "https://registry.npmmirror.com/arch/-/arch-2.2.0.tgz#1bc47818f305764f23ab3306b0bfc086c5a29d11" + integrity sha512-Of/R0wqp83cgHozfIYLbBMnej79U/SVGOOyuB3VVFv1NRM/PSFMK12x9KVtiYzJqmnU5WR2qp0Z5rHb7sWGnFQ== + +are-we-there-yet@~1.1.2: + version "1.1.7" + resolved "https://registry.npmmirror.com/are-we-there-yet/-/are-we-there-yet-1.1.7.tgz#b15474a932adab4ff8a50d9adfa7e4e926f21146" + integrity sha512-nxwy40TuMiUGqMyRHgCSWZ9FM4VAoRP4xUYSTv5ImRog+h9yISPbVH7H8fASCIzYn9wlEv4zvFL7uKDMCFQm3g== + dependencies: + delegates "^1.0.0" + readable-stream "^2.0.6" + +arg@^4.1.0: + version "4.1.3" + resolved "https://registry.npmmirror.com/arg/-/arg-4.1.3.tgz#269fc7ad5b8e42cb63c896d5666017261c144089" + integrity sha512-58S9QDqG0Xx27YwPSt9fJxivjYl432YCwfDMfZ+71RAqUrZef7LrKQZ3LHLOwCS4FLNBplP533Zx895SeOCHvA== + +arg@^5.0.2: + version "5.0.2" + resolved "https://registry.npmmirror.com/arg/-/arg-5.0.2.tgz#c81433cc427c92c4dcf4865142dbca6f15acd59c" + integrity sha512-PYjyFOLKQ9y57JvQ6QLo8dAgNqswh8M1RMJYdQduT6xbWSgK36P/Z/v+p888pM69jMMfS8Xd8F6I1kQ/I9HUGg== + +argparse@^1.0.7: + version "1.0.10" + resolved "https://registry.npmmirror.com/argparse/-/argparse-1.0.10.tgz#bcd6791ea5ae09725e17e5ad988134cd40b3d911" + integrity sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg== + dependencies: + sprintf-js "~1.0.2" + +aria-query@^4.2.2: + version "4.2.2" + resolved "https://registry.npmmirror.com/aria-query/-/aria-query-4.2.2.tgz#0d2ca6c9aceb56b8977e9fed6aed7e15bbd2f83b" + integrity sha512-o/HelwhuKpTj/frsOsbNLNgnNGVIFsVP/SW2BSF14gVl7kAfMOJ6/8wUAUvG1R1NHKrfG+2sHZTu0yauT1qBrA== + dependencies: + "@babel/runtime" "^7.10.2" + "@babel/runtime-corejs3" "^7.10.2" + +arity-n@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/arity-n/-/arity-n-1.0.4.tgz#d9e76b11733e08569c0847ae7b39b2860b30b745" + integrity sha512-fExL2kFDC1Q2DUOx3whE/9KoN66IzkY4b4zUHUBFM1ojEYjZZYDcUW3bek/ufGionX9giIKDC5redH2IlGqcQQ== + +array-flatten@1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/array-flatten/-/array-flatten-1.1.1.tgz#9a5f699051b1e7073328f2a008968b64ea2955d2" + integrity sha512-PCVAQswWemu6UdxsDFFX/+gVeYqKAod3D3UVm91jHwynguOwAvYPhx8nNlM++NqRcK6CxxpUafjmhIdKiHibqg== + +array-includes@^3.1.4, array-includes@^3.1.5: + version "3.1.5" + resolved "https://registry.npmmirror.com/array-includes/-/array-includes-3.1.5.tgz#2c320010db8d31031fd2a5f6b3bbd4b1aad31bdb" + integrity sha512-iSDYZMMyTPkiFasVqfuAQnWAYcvO/SeBSCGKePoEthjp4LEMTe4uLc7b025o4jAZpHhihh8xPo99TNWUWWkGDQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.19.5" + get-intrinsic "^1.1.1" + is-string "^1.0.7" + +array-iterate@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/array-iterate/-/array-iterate-1.1.4.tgz#add1522e9dd9749bb41152d08b845bd08d6af8b7" + integrity sha512-sNRaPGh9nnmdC8Zf+pT3UqP8rnWj5Hf9wiFGsX3wUQ2yVSIhO2ShFwCoceIPpB41QF6i2OEmrHmCo36xronCVA== + +array-tree-filter@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/array-tree-filter/-/array-tree-filter-2.1.0.tgz#873ac00fec83749f255ac8dd083814b4f6329190" + integrity sha512-4ROwICNlNw/Hqa9v+rk5h22KjmzB1JGTMVKP2AKJBOCgb0yL0ASf0+YvCcLNNwquOHNX48jkeZIJ3a+oOQqKcw== + +array-union@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/array-union/-/array-union-2.1.0.tgz#b798420adbeb1de828d84acd8a2e23d3efe85e8d" + integrity sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw== + +array.prototype.flat@^1.2.5: + version "1.3.0" + resolved "https://registry.npmmirror.com/array.prototype.flat/-/array.prototype.flat-1.3.0.tgz#0b0c1567bf57b38b56b4c97b8aa72ab45e4adc7b" + integrity sha512-12IUEkHsAhA4DY5s0FPgNXIdc8VRSqD9Zp78a5au9abH/SOBrsp082JOWFNTjkMozh8mqcdiKuaLGhPeYztxSw== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.2" + es-shim-unscopables "^1.0.0" + +array.prototype.flatmap@^1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/array.prototype.flatmap/-/array.prototype.flatmap-1.3.0.tgz#a7e8ed4225f4788a70cd910abcf0791e76a5534f" + integrity sha512-PZC9/8TKAIxcWKdyeb77EzULHPrIX/tIZebLJUQOMR1OwYosT8yggdfWScfTBCDj5utONvOuPQQumYsU2ULbkg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.2" + es-shim-unscopables "^1.0.0" + +array.prototype.reduce@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/array.prototype.reduce/-/array.prototype.reduce-1.0.4.tgz#8167e80089f78bff70a99e20bd4201d4663b0a6f" + integrity sha512-WnM+AjG/DvLRLo4DDl+r+SvCzYtD2Jd9oeBYMcEaI7t3fFrHY9M53/wdLcTvmZNQ70IU6Htj0emFkZ5TS+lrdw== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.2" + es-array-method-boxes-properly "^1.0.0" + is-string "^1.0.7" + +arrify@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/arrify/-/arrify-1.0.1.tgz#898508da2226f380df904728456849c1501a4b0d" + integrity sha512-3CYzex9M9FGQjCGMGyi6/31c8GJbgb0qGyrx5HWxPd0aCwh4cB2YjMb2Xf9UuoogrMrlO9cTqnB5rI5GHZTcUA== + +arrify@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/arrify/-/arrify-2.0.1.tgz#c9655e9331e0abcd588d2a7cad7e9956f66701fa" + integrity sha512-3duEwti880xqi4eAMN8AyR4a0ByT90zoYdLlevfrvU43vb0YZwZVfxOgxWrLXXXpyugL0hNZc9G6BiB5B3nUug== + +asap@~2.0.3: + version "2.0.6" + resolved "https://registry.npmmirror.com/asap/-/asap-2.0.6.tgz#e50347611d7e690943208bbdafebcbc2fb866d46" + integrity sha512-BSHWgDSAiKs50o2Re8ppvp3seVHXSRM44cdSsT9FfNEUUZLOGWVCsiWaRPWM1Znn+mqZ1OfVZ3z3DWEzSp7hRA== + +asn1@~0.2.3: + version "0.2.6" + resolved "https://registry.npmmirror.com/asn1/-/asn1-0.2.6.tgz#0d3a7bb6e64e02a90c0303b31f292868ea09a08d" + integrity sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ== + dependencies: + safer-buffer "~2.1.0" + +assert-plus@1.0.0, assert-plus@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/assert-plus/-/assert-plus-1.0.0.tgz#f12e0f3c5d77b0b1cdd9146942e4e96c1e4dd525" + integrity sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw== + +ast-types-flow@^0.0.7: + version "0.0.7" + resolved "https://registry.npmmirror.com/ast-types-flow/-/ast-types-flow-0.0.7.tgz#f70b735c6bca1a5c9c22d982c3e39e7feba3bdad" + integrity sha512-eBvWn1lvIApYMhzQMsu9ciLfkBY499mFZlNqG+/9WR7PVlroQw0vG30cOQQbaKz3sCEc44TAOu2ykzqXSNnwag== + +astral-regex@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/astral-regex/-/astral-regex-2.0.0.tgz#483143c567aeed4785759c0865786dc77d7d2e31" + integrity sha512-Z7tMw1ytTXt5jqMcOP+OQteU1VuNK9Y02uuJtKQ1Sv69jXQKKg5cibLwGJow8yzZP+eAc18EmLGPal0bp36rvQ== + +async-cache@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/async-cache/-/async-cache-1.1.0.tgz#4a9a5a89d065ec5d8e5254bd9ee96ba76c532b5a" + integrity sha512-YDQc4vBn5NFhY6g6HhVshyi3Fy9+SQ5ePnE7JLDJn1DoL+i7ER+vMwtTNOYk9leZkYMnOwpBCWqyLDPw8Aig8g== + dependencies: + lru-cache "^4.0.0" + +async-foreach@^0.1.3: + version "0.1.3" + resolved "https://registry.npmmirror.com/async-foreach/-/async-foreach-0.1.3.tgz#36121f845c0578172de419a97dbeb1d16ec34542" + integrity sha512-VUeSMD8nEGBWaZK4lizI1sf3yEC7pnAQ/mrI7pC2fBz2s/tq5jWWEngTwaf0Gruu/OoXRGLGg1XFqpYBiGTYJA== + +async-retry-ng@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/async-retry-ng/-/async-retry-ng-2.0.1.tgz#f5285ec1c52654a2ba6a505d0c18b1eadfaebd41" + integrity sha512-iitlc2murdQ3/A5Re3CcplQBEf7vOmFrFQ6RFn3+/+zZUyIHYkZnnEziMSa6YIb2Bs2EJEPZWReTxjHqvQbDbw== + +async-validator@^4.1.0: + version "4.1.1" + resolved "https://registry.npmmirror.com/async-validator/-/async-validator-4.1.1.tgz#3cd1437faa2de64743f7d56649dd904c946a18fe" + integrity sha512-p4DO/JXwjs8klJyJL8Q2oM4ks5fUTze/h5k10oPPKMiLe1fj3G1QMzPHNmN1Py4ycOk7WlO2DcGXv1qiESJCZA== + +async@1.5.2: + version "1.5.2" + resolved "https://registry.npmmirror.com/async/-/async-1.5.2.tgz#ec6a61ae56480c0c3cb241c95618e20892f9672a" + integrity sha512-nSVgobk4rv61R9PUSDtYt7mPVB2olxNR5RWJcAsH676/ef11bUZwvu7+RGYrYauVdDPcO519v68wRhXQtxsV9w== + +async@^3.2.3: + version "3.2.4" + resolved "https://registry.npmmirror.com/async/-/async-3.2.4.tgz#2d22e00f8cddeb5fde5dd33522b56d1cf569a81c" + integrity sha512-iAB+JbDEGXhyIUavoDl9WP/Jj106Kz9DEn1DPgYw5ruDn0e3Wgi3sKFm55sASdGBNOQB8F59d9qQ7deqrHA8wQ== + +asynckit@^0.4.0: + version "0.4.0" + resolved "https://registry.npmmirror.com/asynckit/-/asynckit-0.4.0.tgz#c79ed97f7f34cb8f2ba1bc9790bcc366474b4b79" + integrity sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q== + +at-least-node@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/at-least-node/-/at-least-node-1.0.0.tgz#602cd4b46e844ad4effc92a8011a3c46e0238dc2" + integrity sha512-+q/t7Ekv1EDY2l6Gda6LLiX14rU9TV20Wa3ofeQmwPFZbOMo9DXrLbOjFaaclkXKWidIaopwAObQDqwWtGUjqg== + +atob@^2.1.2: + version "2.1.2" + resolved "https://registry.npmmirror.com/atob/-/atob-2.1.2.tgz#6d9517eb9e030d2436666651e86bd9f6f13533c9" + integrity sha512-Wm6ukoaOGJi/73p/cl2GvLjTI5JM1k/O14isD73YML8StrH/7/lRFgmg8nICZgD3bZZvjwCGxtMOD3wWNAu8cg== + +auto-bind@~4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/auto-bind/-/auto-bind-4.0.0.tgz#e3589fc6c2da8f7ca43ba9f84fa52a744fc997fb" + integrity sha512-Hdw8qdNiqdJ8LqT0iK0sVzkFbzg6fhnQqqfWhBDxcHZvU75+B+ayzTy8x+k5Ix0Y92XOhOUlx74ps+bA6BeYMQ== + +autoprefixer@^10.4.0, autoprefixer@^10.4.7: + version "10.4.7" + resolved "https://registry.npmmirror.com/autoprefixer/-/autoprefixer-10.4.7.tgz#1db8d195f41a52ca5069b7593be167618edbbedf" + integrity sha512-ypHju4Y2Oav95SipEcCcI5J7CGPuvz8oat7sUtYj3ClK44bldfvtvcxK6IEK++7rqB7YchDGzweZIBG+SD0ZAA== + dependencies: + browserslist "^4.20.3" + caniuse-lite "^1.0.30001335" + fraction.js "^4.2.0" + normalize-range "^0.1.2" + picocolors "^1.0.0" + postcss-value-parser "^4.2.0" + +aws-sign2@~0.7.0: + version "0.7.0" + resolved "https://registry.npmmirror.com/aws-sign2/-/aws-sign2-0.7.0.tgz#b46e890934a9591f2d2f6f86d7e6a9f1b3fe76a8" + integrity sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA== + +aws4@^1.8.0: + version "1.11.0" + resolved "https://registry.npmmirror.com/aws4/-/aws4-1.11.0.tgz#d61f46d83b2519250e2784daf5b09479a8b41c59" + integrity sha512-xh1Rl34h6Fi1DC2WWKfxUTVqRsNnr6LsKz2+hfwDxQJWmrx8+c7ylaqBMcHfl1U1r2dsifOvKX3LQuLNZ+XSvA== + +axe-core@^4.3.5: + version "4.4.2" + resolved "https://registry.npmmirror.com/axe-core/-/axe-core-4.4.2.tgz#dcf7fb6dea866166c3eab33d68208afe4d5f670c" + integrity sha512-LVAaGp/wkkgYJcjmHsoKx4juT1aQvJyPcW09MLCjVTh3V2cc6PnyempiLMNH5iMdfIX/zdbjUx2KDjMLCTdPeA== + +axios@^0.21.1: + version "0.21.4" + resolved "https://registry.npmmirror.com/axios/-/axios-0.21.4.tgz#c67b90dc0568e5c1cf2b0b858c43ba28e2eda575" + integrity sha512-ut5vewkiu8jjGBdqpM44XxjuCjq9LAKeHVmoVfHVzy8eHgxxq8SbAVQNovDA8mVi05kP0Ea/n/UzcSHcTJQfNg== + dependencies: + follow-redirects "^1.14.0" + +axobject-query@^2.2.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/axobject-query/-/axobject-query-2.2.0.tgz#943d47e10c0b704aa42275e20edf3722648989be" + integrity sha512-Td525n+iPOOyUQIeBfcASuG6uJsDOITl7Mds5gFyerkWiX7qhUTdYUBlSgNMyVqtSJqwpt1kXGLdUt6SykLMRA== + +babel-extract-comments@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/babel-extract-comments/-/babel-extract-comments-1.0.0.tgz#0a2aedf81417ed391b85e18b4614e693a0351a21" + integrity sha512-qWWzi4TlddohA91bFwgt6zO/J0X+io7Qp184Fw0m2JYRSTZnJbFR8+07KmzudHCZgOiKRCrjhylwv9Xd8gfhVQ== + dependencies: + babylon "^6.18.0" + +babel-jsx-utils@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/babel-jsx-utils/-/babel-jsx-utils-1.1.0.tgz#304ce4fce0c86cbeee849551a45eb4ed1036381a" + integrity sha512-Mh1j/rw4xM9T3YICkw22aBQ78FhsHdsmlb9NEk4uVAFBOg+Ez9ZgXXHugoBPCZui3XLomk/7/JBBH4daJqTkQQ== + +babel-loader@^8.2.3: + version "8.2.5" + resolved "https://registry.npmmirror.com/babel-loader/-/babel-loader-8.2.5.tgz#d45f585e654d5a5d90f5350a779d7647c5ed512e" + integrity sha512-OSiFfH89LrEMiWd4pLNqGz4CwJDtbs2ZVc+iGu2HrkRfPxId9F2anQj38IxWpmRfsUY0aBZYi1EFcd3mhtRMLQ== + dependencies: + find-cache-dir "^3.3.1" + loader-utils "^2.0.0" + make-dir "^3.1.0" + schema-utils "^2.6.5" + +babel-plugin-add-module-exports@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/babel-plugin-add-module-exports/-/babel-plugin-add-module-exports-1.0.4.tgz#6caa4ddbe1f578c6a5264d4d3e6c8a2720a7ca2b" + integrity sha512-g+8yxHUZ60RcyaUpfNzy56OtWW+x9cyEe9j+CranqLiqbju2yf/Cy6ZtYK40EZxtrdHllzlVZgLmcOUCTlJ7Jg== + +babel-plugin-dynamic-import-node@^2.3.3: + version "2.3.3" + resolved "https://registry.npmmirror.com/babel-plugin-dynamic-import-node/-/babel-plugin-dynamic-import-node-2.3.3.tgz#84fda19c976ec5c6defef57f9427b3def66e17a3" + integrity sha512-jZVI+s9Zg3IqA/kdi0i6UDCybUI3aSBLnglhYbSSjKlV7yF1F/5LWv8MakQmvYpnbJDS6fcBL2KzHSxNCMtWSQ== + dependencies: + object.assign "^4.1.0" + +babel-plugin-lodash@^3.3.4: + version "3.3.4" + resolved "https://registry.npmmirror.com/babel-plugin-lodash/-/babel-plugin-lodash-3.3.4.tgz#4f6844358a1340baed182adbeffa8df9967bc196" + integrity sha512-yDZLjK7TCkWl1gpBeBGmuaDIFhZKmkoL+Cu2MUUjv5VxUZx/z7tBGBCBcQs5RI1Bkz5LLmNdjx7paOyQtMovyg== + dependencies: + "@babel/helper-module-imports" "^7.0.0-beta.49" + "@babel/types" "^7.0.0-beta.49" + glob "^7.1.1" + lodash "^4.17.10" + require-package-name "^2.0.1" + +babel-plugin-macros@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/babel-plugin-macros/-/babel-plugin-macros-3.1.0.tgz#9ef6dc74deb934b4db344dc973ee851d148c50c1" + integrity sha512-Cg7TFGpIr01vOQNODXOOaGz2NpCU5gl8x1qJFbb6hbZxR7XrcE2vtbAsTAbJ7/xwJtUuJEw8K8Zr/AE0LHlesg== + dependencies: + "@babel/runtime" "^7.12.5" + cosmiconfig "^7.0.0" + resolve "^1.19.0" + +babel-plugin-polyfill-corejs2@^0.3.0: + version "0.3.1" + resolved "https://registry.npmmirror.com/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.3.1.tgz#440f1b70ccfaabc6b676d196239b138f8a2cfba5" + integrity sha512-v7/T6EQcNfVLfcN2X8Lulb7DjprieyLWJK/zOWH5DUYcAgex9sP3h25Q+DLsX9TloXe3y1O8l2q2Jv9q8UVB9w== + dependencies: + "@babel/compat-data" "^7.13.11" + "@babel/helper-define-polyfill-provider" "^0.3.1" + semver "^6.1.1" + +babel-plugin-polyfill-corejs3@^0.5.0: + version "0.5.2" + resolved "https://registry.npmmirror.com/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.5.2.tgz#aabe4b2fa04a6e038b688c5e55d44e78cd3a5f72" + integrity sha512-G3uJih0XWiID451fpeFaYGVuxHEjzKTHtc9uGFEjR6hHrvNzeS/PX+LLLcetJcytsB5m4j+K3o/EpXJNb/5IEQ== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.3.1" + core-js-compat "^3.21.0" + +babel-plugin-polyfill-regenerator@^0.3.0: + version "0.3.1" + resolved "https://registry.npmmirror.com/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.3.1.tgz#2c0678ea47c75c8cc2fbb1852278d8fb68233990" + integrity sha512-Y2B06tvgHYt1x0yz17jGkGeeMr5FeKUu+ASJ+N6nB5lQ8Dapfg42i0OVrf8PNGJ3zKL4A23snMi1IRwrqqND7A== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.3.1" + +babel-plugin-remove-graphql-queries@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/babel-plugin-remove-graphql-queries/-/babel-plugin-remove-graphql-queries-4.17.0.tgz#debb5b83b069ccd4f10e33536aad3602157900f0" + integrity sha512-UovjamHhy5y46WyBO/hfqxrLbMq04o+hlTeqCRzeLol4eWtH+8CufdLAZWYtM1l1BIChwvXCxDY2ugeXlbj7+w== + dependencies: + "@babel/runtime" "^7.15.4" + gatsby-core-utils "^3.17.0" + +babel-plugin-syntax-object-rest-spread@^6.8.0: + version "6.13.0" + resolved "https://registry.npmmirror.com/babel-plugin-syntax-object-rest-spread/-/babel-plugin-syntax-object-rest-spread-6.13.0.tgz#fd6536f2bce13836ffa3a5458c4903a597bb3bf5" + integrity sha512-C4Aq+GaAj83pRQ0EFgTvw5YO6T3Qz2KGrNRwIj9mSoNHVvdZY4KO2uA6HNtNXCw993iSZnckY1aLW8nOi8i4+w== + +babel-plugin-syntax-trailing-function-commas@^7.0.0-beta.0: + version "7.0.0-beta.0" + resolved "https://registry.npmmirror.com/babel-plugin-syntax-trailing-function-commas/-/babel-plugin-syntax-trailing-function-commas-7.0.0-beta.0.tgz#aa213c1435e2bffeb6fca842287ef534ad05d5cf" + integrity sha512-Xj9XuRuz3nTSbaTXWv3itLOcxyF4oPD8douBBmj7U9BBC6nEBYfyOJYQMf/8PJAFotC62UY5dFfIGEPr7WswzQ== + +babel-plugin-transform-object-rest-spread@^6.26.0: + version "6.26.0" + resolved "https://registry.npmmirror.com/babel-plugin-transform-object-rest-spread/-/babel-plugin-transform-object-rest-spread-6.26.0.tgz#0f36692d50fef6b7e2d4b3ac1478137a963b7b06" + integrity sha512-ocgA9VJvyxwt+qJB0ncxV8kb/CjfTcECUY4tQ5VT7nP6Aohzobm8CDFaQ5FHdvZQzLmf0sgDxB8iRXZXxwZcyA== + dependencies: + babel-plugin-syntax-object-rest-spread "^6.8.0" + babel-runtime "^6.26.0" + +babel-plugin-transform-react-remove-prop-types@^0.4.24: + version "0.4.24" + resolved "https://registry.npmmirror.com/babel-plugin-transform-react-remove-prop-types/-/babel-plugin-transform-react-remove-prop-types-0.4.24.tgz#f2edaf9b4c6a5fbe5c1d678bfb531078c1555f3a" + integrity sha512-eqj0hVcJUR57/Ug2zE1Yswsw4LhuqqHhD+8v120T1cl3kjg76QwtyBrdIk4WVwK+lAhBJVYCd/v+4nc4y+8JsA== + +babel-preset-fbjs@^3.4.0: + version "3.4.0" + resolved "https://registry.npmmirror.com/babel-preset-fbjs/-/babel-preset-fbjs-3.4.0.tgz#38a14e5a7a3b285a3f3a86552d650dca5cf6111c" + integrity sha512-9ywCsCvo1ojrw0b+XYk7aFvTH6D9064t0RIL1rtMf3nsa02Xw41MS7sZw216Im35xj/UY0PDBQsa1brUDDF1Ow== + dependencies: + "@babel/plugin-proposal-class-properties" "^7.0.0" + "@babel/plugin-proposal-object-rest-spread" "^7.0.0" + "@babel/plugin-syntax-class-properties" "^7.0.0" + "@babel/plugin-syntax-flow" "^7.0.0" + "@babel/plugin-syntax-jsx" "^7.0.0" + "@babel/plugin-syntax-object-rest-spread" "^7.0.0" + "@babel/plugin-transform-arrow-functions" "^7.0.0" + "@babel/plugin-transform-block-scoped-functions" "^7.0.0" + "@babel/plugin-transform-block-scoping" "^7.0.0" + "@babel/plugin-transform-classes" "^7.0.0" + "@babel/plugin-transform-computed-properties" "^7.0.0" + "@babel/plugin-transform-destructuring" "^7.0.0" + "@babel/plugin-transform-flow-strip-types" "^7.0.0" + "@babel/plugin-transform-for-of" "^7.0.0" + "@babel/plugin-transform-function-name" "^7.0.0" + "@babel/plugin-transform-literals" "^7.0.0" + "@babel/plugin-transform-member-expression-literals" "^7.0.0" + "@babel/plugin-transform-modules-commonjs" "^7.0.0" + "@babel/plugin-transform-object-super" "^7.0.0" + "@babel/plugin-transform-parameters" "^7.0.0" + "@babel/plugin-transform-property-literals" "^7.0.0" + "@babel/plugin-transform-react-display-name" "^7.0.0" + "@babel/plugin-transform-react-jsx" "^7.0.0" + "@babel/plugin-transform-shorthand-properties" "^7.0.0" + "@babel/plugin-transform-spread" "^7.0.0" + "@babel/plugin-transform-template-literals" "^7.0.0" + babel-plugin-syntax-trailing-function-commas "^7.0.0-beta.0" + +babel-preset-gatsby@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/babel-preset-gatsby/-/babel-preset-gatsby-2.17.0.tgz#5b158559d1221ff71c95be8a6a60c33d9071d964" + integrity sha512-hLeovvehdKkfFAsfTKCATKQasLyQa49TrJtct5VthERrb2dhkV71NIfmbiEa9Ae5x+qGYhYvufqMROGptewXbQ== + dependencies: + "@babel/plugin-proposal-class-properties" "^7.14.0" + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.14.5" + "@babel/plugin-proposal-optional-chaining" "^7.14.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-transform-classes" "^7.15.4" + "@babel/plugin-transform-runtime" "^7.15.0" + "@babel/plugin-transform-spread" "^7.14.6" + "@babel/preset-env" "^7.15.4" + "@babel/preset-react" "^7.14.0" + "@babel/runtime" "^7.15.4" + babel-plugin-dynamic-import-node "^2.3.3" + babel-plugin-macros "^3.1.0" + babel-plugin-transform-react-remove-prop-types "^0.4.24" + gatsby-core-utils "^3.17.0" + gatsby-legacy-polyfills "^2.17.0" + +babel-runtime@^6.26.0: + version "6.26.0" + resolved "https://registry.npmmirror.com/babel-runtime/-/babel-runtime-6.26.0.tgz#965c7058668e82b55d7bfe04ff2337bc8b5647fe" + integrity sha512-ITKNuq2wKlW1fJg9sSW52eepoYgZBggvOAHC0u/CYu/qxQ9EVzThCgR69BnSXLHjy2f7SY5zaQ4yt7H9ZVxY2g== + dependencies: + core-js "^2.4.0" + regenerator-runtime "^0.11.0" + +babylon@^6.18.0: + version "6.18.0" + resolved "https://registry.npmmirror.com/babylon/-/babylon-6.18.0.tgz#af2f3b88fa6f5c1e4c634d1a0f8eac4f55b395e3" + integrity sha512-q/UEjfGJ2Cm3oKV71DJz9d25TPnq5rhBVL2Q4fA5wcC3jcrdn7+SssEybFIxwAvvP+YCsCYNKughoF33GxgycQ== + +backo2@^1.0.2, backo2@~1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/backo2/-/backo2-1.0.2.tgz#31ab1ac8b129363463e35b3ebb69f4dfcfba7947" + integrity sha512-zj6Z6M7Eq+PBZ7PQxl5NT665MvJdAkzp0f60nAJ+sLaSCBPMwVak5ZegFbgVCzFcCJTKFoMizvM5Ld7+JrRJHA== + +bail@^1.0.0: + version "1.0.5" + resolved "https://registry.npmmirror.com/bail/-/bail-1.0.5.tgz#b6fa133404a392cbc1f8c4bf63f5953351e7a776" + integrity sha512-xFbRxM1tahm08yHBP16MMjVUAvDaBMD38zsM9EMAUN61omwLmKlOpB/Zku5QkjZ8TZ4vn53pj+t518cH0S03RQ== + +balanced-match@^1.0.0: + version "1.0.2" + resolved "https://registry.npmmirror.com/balanced-match/-/balanced-match-1.0.2.tgz#e83e3a7e3f300b34cb9d87f615fa0cbf357690ee" + integrity sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw== + +base-x@^3.0.8: + version "3.0.9" + resolved "https://registry.npmmirror.com/base-x/-/base-x-3.0.9.tgz#6349aaabb58526332de9f60995e548a53fe21320" + integrity sha512-H7JU6iBHTal1gp56aKoaa//YUxEaAOUiydvrV/pILqIHXTtqxSkATOnDA2u+jZ/61sD+L/412+7kzXRtWukhpQ== + dependencies: + safe-buffer "^5.0.1" + +base16@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/base16/-/base16-1.0.0.tgz#e297f60d7ec1014a7a971a39ebc8a98c0b681e70" + integrity sha512-pNdYkNPiJUnEhnfXV56+sQy8+AaPcG3POZAUnwr4EeqCUZFz4u2PePbo3e5Gj4ziYPCWGUZT9RHisvJKnwFuBQ== + +base64-arraybuffer@0.1.4: + version "0.1.4" + resolved "https://registry.npmmirror.com/base64-arraybuffer/-/base64-arraybuffer-0.1.4.tgz#9818c79e059b1355f97e0428a017c838e90ba812" + integrity sha512-a1eIFi4R9ySrbiMuyTGx5e92uRH5tQY6kArNcFaKBUleIoLjdjBg7Zxm3Mqm3Kmkf27HLR/1fnxX9q8GQ7Iavg== + +base64-js@^1.3.1: + version "1.5.1" + resolved "https://registry.npmmirror.com/base64-js/-/base64-js-1.5.1.tgz#1b1b440160a5bf7ad40b650f095963481903930a" + integrity sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA== + +base64id@2.0.0, base64id@~2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/base64id/-/base64id-2.0.0.tgz#2770ac6bc47d312af97a8bf9a634342e0cd25cb6" + integrity sha512-lGe34o6EHj9y3Kts9R4ZYs/Gr+6N7MCaMlIFA3F1R2O5/m7K06AxfSeO5530PEERE6/WyEg3lsuyw4GHlPZHog== + +bcrypt-pbkdf@^1.0.0: + version "1.0.2" + resolved "https://registry.npmmirror.com/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz#a4301d389b6a43f9b67ff3ca11a3f6637e360e9e" + integrity sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w== + dependencies: + tweetnacl "^0.14.3" + +better-opn@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/better-opn/-/better-opn-2.1.1.tgz#94a55b4695dc79288f31d7d0e5f658320759f7c6" + integrity sha512-kIPXZS5qwyKiX/HcRvDYfmBQUa8XP17I0mYZZ0y4UhpYOSvtsLHDYqmomS+Mj20aDvD3knEiQ0ecQy2nhio3yA== + dependencies: + open "^7.0.3" + +big.js@^5.2.2: + version "5.2.2" + resolved "https://registry.npmmirror.com/big.js/-/big.js-5.2.2.tgz#65f0af382f578bcdc742bd9c281e9cb2d7768328" + integrity sha512-vyL2OymJxmarO8gxMr0mhChsO9QGwhynfuu4+MHTAW6czfq9humCB7rKpUjDd9YUiDPU4mzpyupFSvOClAwbmQ== + +binary-extensions@^2.0.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/binary-extensions/-/binary-extensions-2.2.0.tgz#75f502eeaf9ffde42fc98829645be4ea76bd9e2d" + integrity sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA== + +bl@^4.0.0, bl@^4.0.3: + version "4.1.0" + resolved "https://registry.npmmirror.com/bl/-/bl-4.1.0.tgz#451535264182bec2fbbc83a62ab98cf11d9f7b3a" + integrity sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w== + dependencies: + buffer "^5.5.0" + inherits "^2.0.4" + readable-stream "^3.4.0" + +bluebird@^3.7.2: + version "3.7.2" + resolved "https://registry.npmmirror.com/bluebird/-/bluebird-3.7.2.tgz#9f229c15be272454ffa973ace0dbee79a1b0c36f" + integrity sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg== + +bmp-js@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/bmp-js/-/bmp-js-0.1.0.tgz#e05a63f796a6c1ff25f4771ec7adadc148c07233" + integrity sha512-vHdS19CnY3hwiNdkaqk93DvjVLfbEcI8mys4UjuWrlX1haDmroo8o4xCzh4wD6DGV6HxRCyauwhHRqMTfERtjw== + +body-parser@1.20.0, body-parser@^1.19.0: + version "1.20.0" + resolved "https://registry.npmmirror.com/body-parser/-/body-parser-1.20.0.tgz#3de69bd89011c11573d7bfee6a64f11b6bd27cc5" + integrity sha512-DfJ+q6EPcGKZD1QWUjSpqp+Q7bDQTsQIF4zfUAtZ6qk+H/3/QRhg9CEp39ss+/T2vw0+HaidC0ecJj/DRLIaKg== + dependencies: + bytes "3.1.2" + content-type "~1.0.4" + debug "2.6.9" + depd "2.0.0" + destroy "1.2.0" + http-errors "2.0.0" + iconv-lite "0.4.24" + on-finished "2.4.1" + qs "6.10.3" + raw-body "2.5.1" + type-is "~1.6.18" + unpipe "1.0.0" + +boolbase@^1.0.0, boolbase@~1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/boolbase/-/boolbase-1.0.0.tgz#68dff5fbe60c51eb37725ea9e3ed310dcc1e776e" + integrity sha512-JZOSA7Mo9sNGB8+UjSgzdLtokWAky1zbztM3WRLCbZ70/3cTANmQmOdR7y2g+J0e2WXywy1yS468tY+IruqEww== + +boxen@^4.2.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/boxen/-/boxen-4.2.0.tgz#e411b62357d6d6d36587c8ac3d5d974daa070e64" + integrity sha512-eB4uT9RGzg2odpER62bBwSLvUeGC+WbRjjyyFhGsKnc8wp/m0+hQsMUvUe3H2V0D5vw0nBdO1hCJoZo5mKeuIQ== + dependencies: + ansi-align "^3.0.0" + camelcase "^5.3.1" + chalk "^3.0.0" + cli-boxes "^2.2.0" + string-width "^4.1.0" + term-size "^2.1.0" + type-fest "^0.8.1" + widest-line "^3.1.0" + +boxen@^5.0.0, boxen@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/boxen/-/boxen-5.1.2.tgz#788cb686fc83c1f486dfa8a40c68fc2b831d2b50" + integrity sha512-9gYgQKXx+1nP8mP7CzFyaUARhg7D3n1dF/FnErWmu9l6JvGpNUN278h0aSb+QjoiKSWG+iZ3uHrcqk0qrY9RQQ== + dependencies: + ansi-align "^3.0.0" + camelcase "^6.2.0" + chalk "^4.1.0" + cli-boxes "^2.2.1" + string-width "^4.2.2" + type-fest "^0.20.2" + widest-line "^3.1.0" + wrap-ansi "^7.0.0" + +brace-expansion@^1.1.7: + version "1.1.11" + resolved "https://registry.npmmirror.com/brace-expansion/-/brace-expansion-1.1.11.tgz#3c7fcbf529d87226f3d2f52b966ff5271eb441dd" + integrity sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA== + dependencies: + balanced-match "^1.0.0" + concat-map "0.0.1" + +braces@^3.0.2, braces@~3.0.2: + version "3.0.2" + resolved "https://registry.npmmirror.com/braces/-/braces-3.0.2.tgz#3454e1a462ee8d599e236df336cd9ea4f8afe107" + integrity sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A== + dependencies: + fill-range "^7.0.1" + +browserslist@^4.0.0, browserslist@^4.14.5, browserslist@^4.16.3, browserslist@^4.16.6, browserslist@^4.17.5, browserslist@^4.18.1, browserslist@^4.20.2, browserslist@^4.20.3, browserslist@^4.20.4, browserslist@^4.6.6: + version "4.20.4" + resolved "https://registry.npmmirror.com/browserslist/-/browserslist-4.20.4.tgz#98096c9042af689ee1e0271333dbc564b8ce4477" + integrity sha512-ok1d+1WpnU24XYN7oC3QWgTyMhY/avPJ/r9T00xxvUOIparA/gc+UPUMaod3i+G6s+nI2nUb9xZ5k794uIwShw== + dependencies: + caniuse-lite "^1.0.30001349" + electron-to-chromium "^1.4.147" + escalade "^3.1.1" + node-releases "^2.0.5" + picocolors "^1.0.0" + +bser@2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/bser/-/bser-2.1.1.tgz#e6787da20ece9d07998533cfd9de6f5c38f4bc05" + integrity sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ== + dependencies: + node-int64 "^0.4.0" + +buffer-equal@0.0.1: + version "0.0.1" + resolved "https://registry.npmmirror.com/buffer-equal/-/buffer-equal-0.0.1.tgz#91bc74b11ea405bc916bc6aa908faafa5b4aac4b" + integrity sha512-RgSV6InVQ9ODPdLWJ5UAqBqJBOg370Nz6ZQtRzpt6nUjc8v0St97uJ4PYC6NztqIScrAXafKM3mZPMygSe1ggA== + +buffer-from@^1.0.0: + version "1.1.2" + resolved "https://registry.npmmirror.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5" + integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ== + +buffer-from@~0.1.1: + version "0.1.2" + resolved "https://registry.npmmirror.com/buffer-from/-/buffer-from-0.1.2.tgz#15f4b9bcef012044df31142c14333caf6e0260d0" + integrity sha512-RiWIenusJsmI2KcvqQABB83tLxCByE3upSP8QU3rJDMVFGPWLvPQJt/O1Su9moRWeH7d+Q2HYb68f6+v+tw2vg== + +buffer@^5.2.0, buffer@^5.5.0, buffer@^5.7.0: + version "5.7.1" + resolved "https://registry.npmmirror.com/buffer/-/buffer-5.7.1.tgz#ba62e7c13133053582197160851a8f648e99eed0" + integrity sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ== + dependencies: + base64-js "^1.3.1" + ieee754 "^1.1.13" + +busboy@^0.2.11: + version "0.2.14" + resolved "https://registry.npmmirror.com/busboy/-/busboy-0.2.14.tgz#6c2a622efcf47c57bbbe1e2a9c37ad36c7925453" + integrity sha512-InWFDomvlkEj+xWLBfU3AvnbVYqeTWmQopiW0tWWEy5yehYm2YkGEc59sUmw/4ty5Zj/b0WHGs1LgecuBSBGrg== + dependencies: + dicer "0.2.5" + readable-stream "1.1.x" + +bytes@3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/bytes/-/bytes-3.0.0.tgz#d32815404d689699f85a4ea4fa8755dd13a96048" + integrity sha512-pMhOfFDPiv9t5jjIXkHosWmkSyQbvsgEVNkz0ERHbuLh2T/7j4Mqqpz523Fe8MVY89KC6Sh/QfS2sM+SjgFDcw== + +bytes@3.1.2: + version "3.1.2" + resolved "https://registry.npmmirror.com/bytes/-/bytes-3.1.2.tgz#8b0beeb98605adf1b128fa4386403c009e0221a5" + integrity sha512-/Nf7TyzTx6S3yRJObOAV7956r8cr2+Oj8AC5dt8wSP3BQAoeX58NoHyCU8P8zGkNXStjTSi6fzO6F0pBdcYbEg== + +cache-manager@^2.11.1: + version "2.11.1" + resolved "https://registry.npmmirror.com/cache-manager/-/cache-manager-2.11.1.tgz#212e8c3db15288af653b029a1d9fe12f1fd9df61" + integrity sha512-XhUuc9eYwkzpK89iNewFwtvcDYMUsvtwzHeyEOPJna/WsVsXcrzsA1ft2M0QqPNunEzLhNCYPo05tEfG+YuNow== + dependencies: + async "1.5.2" + lodash.clonedeep "4.5.0" + lru-cache "4.0.0" + +cacheable-lookup@^5.0.3: + version "5.0.4" + resolved "https://registry.npmmirror.com/cacheable-lookup/-/cacheable-lookup-5.0.4.tgz#5a6b865b2c44357be3d5ebc2a467b032719a7005" + integrity sha512-2/kNscPhpcxrOigMZzbiWF7dz8ilhb/nIHU3EyZiXWXpeq/au8qJ8VhdftMkty3n7Gj6HIGalQG8oiBNB3AJgA== + +cacheable-request@^6.0.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/cacheable-request/-/cacheable-request-6.1.0.tgz#20ffb8bd162ba4be11e9567d823db651052ca912" + integrity sha512-Oj3cAGPCqOZX7Rz64Uny2GYAZNliQSqfbePrgAQ1wKAihYmCUnraBtJtKcGR4xz7wF+LoJC+ssFZvv5BgF9Igg== + dependencies: + clone-response "^1.0.2" + get-stream "^5.1.0" + http-cache-semantics "^4.0.0" + keyv "^3.0.0" + lowercase-keys "^2.0.0" + normalize-url "^4.1.0" + responselike "^1.0.2" + +cacheable-request@^7.0.2: + version "7.0.2" + resolved "https://registry.npmmirror.com/cacheable-request/-/cacheable-request-7.0.2.tgz#ea0d0b889364a25854757301ca12b2da77f91d27" + integrity sha512-pouW8/FmiPQbuGpkXQ9BAPv/Mo5xDGANgSNXzTzJ8DrKGuXOssM4wIQRjfanNRh3Yu5cfYPvcorqbhg2KIJtew== + dependencies: + clone-response "^1.0.2" + get-stream "^5.1.0" + http-cache-semantics "^4.0.0" + keyv "^4.0.0" + lowercase-keys "^2.0.0" + normalize-url "^6.0.1" + responselike "^2.0.0" + +call-bind@^1.0.0, call-bind@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/call-bind/-/call-bind-1.0.2.tgz#b1d4e89e688119c3c9a903ad30abb2f6a919be3c" + integrity sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA== + dependencies: + function-bind "^1.1.1" + get-intrinsic "^1.0.2" + +callsites@^3.0.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" + integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== + +camel-case@4.1.2, camel-case@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/camel-case/-/camel-case-4.1.2.tgz#9728072a954f805228225a6deea6b38461e1bd5a" + integrity sha512-gxGWBrTT1JuMx6R+o5PTXMmUnhnVzLQ9SNutD4YqKtI6ap897t3tKECYla6gCWEkplXnlNybEkZg9GEGxKFCgw== + dependencies: + pascal-case "^3.1.2" + tslib "^2.0.3" + +camelcase-css@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/camelcase-css/-/camelcase-css-2.0.1.tgz#ee978f6947914cc30c6b44741b6ed1df7f043fd5" + integrity sha512-QOSvevhslijgYwRx6Rv7zKdMF8lbRmx+uQGx2+vDc+KI/eBnsy9kit5aj23AgGu3pa4t9AgwbnXWqS+iOY+2aA== + +camelcase-keys@^6.2.2: + version "6.2.2" + resolved "https://registry.npmmirror.com/camelcase-keys/-/camelcase-keys-6.2.2.tgz#5e755d6ba51aa223ec7d3d52f25778210f9dc3c0" + integrity sha512-YrwaA0vEKazPBkn0ipTiMpSajYDSe+KjQfrjhcBMxJt/znbvlHd8Pw/Vamaz5EB4Wfhs3SUR3Z9mwRu/P3s3Yg== + dependencies: + camelcase "^5.3.1" + map-obj "^4.0.0" + quick-lru "^4.0.1" + +camelcase@5.3.1, camelcase@^5.0.0, camelcase@^5.3.1: + version "5.3.1" + resolved "https://registry.npmmirror.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" + integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== + +camelcase@^6.2.0: + version "6.3.0" + resolved "https://registry.npmmirror.com/camelcase/-/camelcase-6.3.0.tgz#5685b95eb209ac9c0c177467778c9c84df58ba9a" + integrity sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA== + +caniuse-api@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/caniuse-api/-/caniuse-api-3.0.0.tgz#5e4d90e2274961d46291997df599e3ed008ee4c0" + integrity sha512-bsTwuIg/BZZK/vreVTYYbSWoe2F+71P7K5QGEX+pT250DZbfU1MQ5prOKpPR+LL6uWKK3KMwMCAS74QB3Um1uw== + dependencies: + browserslist "^4.0.0" + caniuse-lite "^1.0.0" + lodash.memoize "^4.1.2" + lodash.uniq "^4.5.0" + +caniuse-lite@^1.0.0, caniuse-lite@^1.0.30001335, caniuse-lite@^1.0.30001349: + version "1.0.30001354" + resolved "https://registry.npmmirror.com/caniuse-lite/-/caniuse-lite-1.0.30001354.tgz#95c5efdb64148bb4870771749b9a619304755ce5" + integrity sha512-mImKeCkyGDAHNywYFA4bqnLAzTUvVkqPvhY4DV47X+Gl2c5Z8c3KNETnXp14GQt11LvxE8AwjzGxJ+rsikiOzg== + +capital-case@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/capital-case/-/capital-case-1.0.4.tgz#9d130292353c9249f6b00fa5852bee38a717e669" + integrity sha512-ds37W8CytHgwnhGGTi88pcPyR15qoNkOpYwmMMfnWqqWgESapLqvDx6huFjQ5vqWSn2Z06173XNA7LtMOeUh1A== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + upper-case-first "^2.0.2" + +caseless@~0.12.0: + version "0.12.0" + resolved "https://registry.npmmirror.com/caseless/-/caseless-0.12.0.tgz#1b681c21ff84033c826543090689420d187151dc" + integrity sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw== + +ccount@^1.0.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/ccount/-/ccount-1.1.0.tgz#246687debb6014735131be8abab2d93898f8d043" + integrity sha512-vlNK021QdI7PNeiUh/lKkC/mNHHfV0m/Ad5JoI0TYtlBnJAslM/JIkm/tGC88bkLIwO6OQ5uV6ztS6kVAtCDlg== + +chalk@^1.1.1: + version "1.1.3" + resolved "https://registry.npmmirror.com/chalk/-/chalk-1.1.3.tgz#a8115c55e4a702fe4d150abd3872822a7e09fc98" + integrity sha512-U3lRVLMSlsCfjqYPbLyVv11M9CPW4I728d6TCKMAOJueEeB9/8o+eSsMnxPJD+Q+K909sdESg7C+tIkoH6on1A== + dependencies: + ansi-styles "^2.2.1" + escape-string-regexp "^1.0.2" + has-ansi "^2.0.0" + strip-ansi "^3.0.0" + supports-color "^2.0.0" + +chalk@^2.0.0, chalk@^2.4.1, chalk@^2.4.2: + version "2.4.2" + resolved "https://registry.npmmirror.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" + integrity sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ== + dependencies: + ansi-styles "^3.2.1" + escape-string-regexp "^1.0.5" + supports-color "^5.3.0" + +chalk@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/chalk/-/chalk-3.0.0.tgz#3f73c2bf526591f574cc492c51e2456349f844e4" + integrity sha512-4D3B6Wf41KOYRFdszmDqMCGq5VV/uMAB273JILmO+3jAlh8X4qDtdtgCR3fxtbLEMzSx22QdhnDcJvu2u1fVwg== + dependencies: + ansi-styles "^4.1.0" + supports-color "^7.1.0" + +chalk@^4.0, chalk@^4.0.0, chalk@^4.1.0, chalk@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/chalk/-/chalk-4.1.2.tgz#aac4e2b7734a740867aeb16bf02aad556a1e7a01" + integrity sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA== + dependencies: + ansi-styles "^4.1.0" + supports-color "^7.1.0" + +change-case-all@1.0.14: + version "1.0.14" + resolved "https://registry.npmmirror.com/change-case-all/-/change-case-all-1.0.14.tgz#bac04da08ad143278d0ac3dda7eccd39280bfba1" + integrity sha512-CWVm2uT7dmSHdO/z1CXT/n47mWonyypzBbuCy5tN7uMg22BsfkhwT6oHmFCAk+gL1LOOxhdbB9SZz3J1KTY3gA== + dependencies: + change-case "^4.1.2" + is-lower-case "^2.0.2" + is-upper-case "^2.0.2" + lower-case "^2.0.2" + lower-case-first "^2.0.2" + sponge-case "^1.0.1" + swap-case "^2.0.2" + title-case "^3.0.3" + upper-case "^2.0.2" + upper-case-first "^2.0.2" + +change-case@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/change-case/-/change-case-4.1.2.tgz#fedfc5f136045e2398c0410ee441f95704641e12" + integrity sha512-bSxY2ws9OtviILG1EiY5K7NNxkqg/JnRnFxLtKQ96JaviiIxi7djMrSd0ECT9AC+lttClmYwKw53BWpOMblo7A== + dependencies: + camel-case "^4.1.2" + capital-case "^1.0.4" + constant-case "^3.0.4" + dot-case "^3.0.4" + header-case "^2.0.4" + no-case "^3.0.4" + param-case "^3.0.4" + pascal-case "^3.1.2" + path-case "^3.0.4" + sentence-case "^3.0.4" + snake-case "^3.0.4" + tslib "^2.0.3" + +character-entities-html4@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/character-entities-html4/-/character-entities-html4-1.1.4.tgz#0e64b0a3753ddbf1fdc044c5fd01d0199a02e125" + integrity sha512-HRcDxZuZqMx3/a+qrzxdBKBPUpxWEq9xw2OPZ3a/174ihfrQKVsFhqtthBInFy1zZ9GgZyFXOatNujm8M+El3g== + +character-entities-legacy@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/character-entities-legacy/-/character-entities-legacy-1.1.4.tgz#94bc1845dce70a5bb9d2ecc748725661293d8fc1" + integrity sha512-3Xnr+7ZFS1uxeiUDvV02wQ+QDbc55o97tIV5zHScSPJpcLm/r0DFPcoY3tYRp+VZukxuMeKgXYmsXQHO05zQeA== + +character-entities@^1.0.0: + version "1.2.4" + resolved "https://registry.npmmirror.com/character-entities/-/character-entities-1.2.4.tgz#e12c3939b7eaf4e5b15e7ad4c5e28e1d48c5b16b" + integrity sha512-iBMyeEHxfVnIakwOuDXpVkc54HijNgCyQB2w0VfGQThle6NXn50zU6V/u+LDhxHcDUPojn6Kpga3PTAD8W1bQw== + +character-reference-invalid@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/character-reference-invalid/-/character-reference-invalid-1.1.4.tgz#083329cda0eae272ab3dbbf37e9a382c13af1560" + integrity sha512-mKKUkUbhPpQlCOfIuZkvSEgktjPFIsZKRRbC6KWVEMvlzblj3i3asQv5ODsrwt0N3pHAEvjP8KTQPHkp0+6jOg== + +chardet@^0.7.0: + version "0.7.0" + resolved "https://registry.npmmirror.com/chardet/-/chardet-0.7.0.tgz#90094849f0937f2eedc2425d0d28a9e5f0cbad9e" + integrity sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA== + +cheerio-select@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/cheerio-select/-/cheerio-select-2.1.0.tgz#4d8673286b8126ca2a8e42740d5e3c4884ae21b4" + integrity sha512-9v9kG0LvzrlcungtnJtpGNxY+fzECQKhK4EGJX2vByejiMX84MFNQw4UxPJl3bFbTMw+Dfs37XaIkCwTZfLh4g== + dependencies: + boolbase "^1.0.0" + css-select "^5.1.0" + css-what "^6.1.0" + domelementtype "^2.3.0" + domhandler "^5.0.3" + domutils "^3.0.1" + +cheerio@^1.0.0-rc.10: + version "1.0.0-rc.11" + resolved "https://registry.npmmirror.com/cheerio/-/cheerio-1.0.0-rc.11.tgz#1be84be1a126958366bcc57a11648cd9b30a60c2" + integrity sha512-bQwNaDIBKID5ts/DsdhxrjqFXYfLw4ste+wMKqWA8DyKcS4qwsPP4Bk8ZNaTJjvpiX/qW3BT4sU7d6Bh5i+dag== + dependencies: + cheerio-select "^2.1.0" + dom-serializer "^2.0.0" + domhandler "^5.0.3" + domutils "^3.0.1" + htmlparser2 "^8.0.1" + parse5 "^7.0.0" + parse5-htmlparser2-tree-adapter "^7.0.0" + tslib "^2.4.0" + +chokidar@^3.4.2, chokidar@^3.5.2, chokidar@^3.5.3: + version "3.5.3" + resolved "https://registry.npmmirror.com/chokidar/-/chokidar-3.5.3.tgz#1cf37c8707b932bd1af1ae22c0432e2acd1903bd" + integrity sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw== + dependencies: + anymatch "~3.1.2" + braces "~3.0.2" + glob-parent "~5.1.2" + is-binary-path "~2.1.0" + is-glob "~4.0.1" + normalize-path "~3.0.0" + readdirp "~3.6.0" + optionalDependencies: + fsevents "~2.3.2" + +chownr@^1.1.1: + version "1.1.4" + resolved "https://registry.npmmirror.com/chownr/-/chownr-1.1.4.tgz#6fc9d7b42d32a583596337666e7d08084da2cc6b" + integrity sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg== + +chownr@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/chownr/-/chownr-2.0.0.tgz#15bfbe53d2eab4cf70f18a8cd68ebe5b3cb1dece" + integrity sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ== + +chrome-trace-event@^1.0.2: + version "1.0.3" + resolved "https://registry.npmmirror.com/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz#1015eced4741e15d06664a957dbbf50d041e26ac" + integrity sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg== + +ci-info@2.0.0, ci-info@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/ci-info/-/ci-info-2.0.0.tgz#67a9e964be31a51e15e5010d58e6f12834002f46" + integrity sha512-5tK7EtrZ0N+OLFMthtqOj4fI2Jeb88C4CAZPu25LDVUgXJ0A3Js4PMGqrn0JU1W0Mh1/Z8wZzYPxqUrXeBboCQ== + +classlist-polyfill@^1.0.3: + version "1.2.0" + resolved "https://registry.npmmirror.com/classlist-polyfill/-/classlist-polyfill-1.2.0.tgz#935bc2dfd9458a876b279617514638bcaa964a2e" + integrity sha512-GzIjNdcEtH4ieA2S8NmrSxv7DfEV5fmixQeyTmqmRmRJPGpRBaSnA2a0VrCjyT8iW8JjEdMbKzDotAJf+ajgaQ== + +classnames@2.x, classnames@^2.2.1, classnames@^2.2.3, classnames@^2.2.5, classnames@^2.2.6, classnames@^2.3.1: + version "2.3.1" + resolved "https://registry.npmmirror.com/classnames/-/classnames-2.3.1.tgz#dfcfa3891e306ec1dad105d0e88f4417b8535e8e" + integrity sha512-OlQdbZ7gLfGarSqxesMesDa5uz7KFbID8Kpq/SxIoNGDqY8lSYs0D+hhtBXhcdB3rcbXArFr7vlHheLk1voeNA== + +cli-boxes@^2.2.0, cli-boxes@^2.2.1: + version "2.2.1" + resolved "https://registry.npmmirror.com/cli-boxes/-/cli-boxes-2.2.1.tgz#ddd5035d25094fce220e9cab40a45840a440318f" + integrity sha512-y4coMcylgSCdVinjiDBuR8PCC2bLjyGTwEmPb9NHR/QaNU6EUOXcTY/s6VjGMD6ENSEaeQYHCY0GNGS5jfMwPw== + +cli-cursor@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/cli-cursor/-/cli-cursor-3.1.0.tgz#264305a7ae490d1d03bf0c9ba7c925d1753af307" + integrity sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw== + dependencies: + restore-cursor "^3.1.0" + +cli-width@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/cli-width/-/cli-width-3.0.0.tgz#a2f48437a2caa9a22436e794bf071ec9e61cedf6" + integrity sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw== + +clipboardy@^2.3.0: + version "2.3.0" + resolved "https://registry.npmmirror.com/clipboardy/-/clipboardy-2.3.0.tgz#3c2903650c68e46a91b388985bc2774287dba290" + integrity sha512-mKhiIL2DrQIsuXMgBgnfEHOZOryC7kY7YO//TN6c63wlEm3NG5tz+YgY5rVi29KCmq/QQjKYvM7a19+MDOTHOQ== + dependencies: + arch "^2.1.1" + execa "^1.0.0" + is-wsl "^2.1.1" + +cliui@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/cliui/-/cliui-5.0.0.tgz#deefcfdb2e800784aa34f46fa08e06851c7bbbc5" + integrity sha512-PYeGSEmmHM6zvoef2w8TPzlrnNpXIjTipYK780YswmIP9vjxmd6Y2a3CB2Ks6/AU8NHjZugXvo8w3oWM2qnwXA== + dependencies: + string-width "^3.1.0" + strip-ansi "^5.2.0" + wrap-ansi "^5.1.0" + +cliui@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/cliui/-/cliui-6.0.0.tgz#511d702c0c4e41ca156d7d0e96021f23e13225b1" + integrity sha512-t6wbgtoCXvAzst7QgXxJYqPt0usEfbgQdftEPbLL/cvv6HPE5VgvqCuAIDR0NgU52ds6rFwqrgakNLrHEjCbrQ== + dependencies: + string-width "^4.2.0" + strip-ansi "^6.0.0" + wrap-ansi "^6.2.0" + +clone-deep@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/clone-deep/-/clone-deep-4.0.1.tgz#c19fd9bdbbf85942b4fd979c84dcf7d5f07c2387" + integrity sha512-neHB9xuzh/wk0dIHweyAXv2aPGZIVk3pLMe+/RNzINf17fe0OG96QroktYAUm7SM1PBnzTabaLboqqxDyMU+SQ== + dependencies: + is-plain-object "^2.0.4" + kind-of "^6.0.2" + shallow-clone "^3.0.0" + +clone-response@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/clone-response/-/clone-response-1.0.2.tgz#d1dc973920314df67fbeb94223b4ee350239e96b" + integrity sha512-yjLXh88P599UOyPTFX0POsd7WxnbsVsGohcwzHOLspIhhpalPw1BcqED8NblyZLKcGrL8dTgMlcaZxV2jAD41Q== + dependencies: + mimic-response "^1.0.0" + +clone@^2.1.1: + version "2.1.2" + resolved "https://registry.npmmirror.com/clone/-/clone-2.1.2.tgz#1b7f4b9f591f1e8f83670401600345a02887435f" + integrity sha512-3Pe/CF1Nn94hyhIYpjtiLhdCoEoz0DqQ+988E9gmeEdQZlojxnOb74wctFyuwWQHzqyf9X7C7MG8juUpqBJT8w== + +clsx@^1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/clsx/-/clsx-1.1.1.tgz#98b3134f9abbdf23b2663491ace13c5c03a73188" + integrity sha512-6/bPho624p3S2pMyvP5kKBPXnI3ufHLObBFCfgx+LkeR5lg2XYy2hqZqUf45ypD8COn2bhgGJSUE+l5dhNBieA== + +coa@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/coa/-/coa-2.0.2.tgz#43f6c21151b4ef2bf57187db0d73de229e3e7ec3" + integrity sha512-q5/jG+YQnSy4nRTV4F7lPepBJZ8qBNJJDBuJdoejDyLXgmL7IEo+Le2JDZudFTFt7mrCqIRaSjws4ygRCTCAXA== + dependencies: + "@types/q" "^1.5.1" + chalk "^2.4.1" + q "^1.1.2" + +code-point-at@^1.0.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/code-point-at/-/code-point-at-1.1.0.tgz#0d070b4d043a5bea33a2f1a40e2edb3d9a4ccf77" + integrity sha512-RpAVKQA5T63xEj6/giIbUEtZwJ4UFIc3ZtvEkiaUERylqe8xb5IvqcgOurZLahv93CLKfxcw5YI+DZcUBRyLXA== + +color-convert@^1.9.0: + version "1.9.3" + resolved "https://registry.npmmirror.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" + integrity sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg== + dependencies: + color-name "1.1.3" + +color-convert@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/color-convert/-/color-convert-2.0.1.tgz#72d3a68d598c9bdb3af2ad1e84f21d896abd4de3" + integrity sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ== + dependencies: + color-name "~1.1.4" + +color-name@1.1.3: + version "1.1.3" + resolved "https://registry.npmmirror.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25" + integrity sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw== + +color-name@^1.0.0, color-name@^1.1.4, color-name@~1.1.4: + version "1.1.4" + resolved "https://registry.npmmirror.com/color-name/-/color-name-1.1.4.tgz#c2a09a87acbde69543de6f63fa3995c826c536a2" + integrity sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA== + +color-string@^1.9.0: + version "1.9.1" + resolved "https://registry.npmmirror.com/color-string/-/color-string-1.9.1.tgz#4467f9146f036f855b764dfb5bf8582bf342c7a4" + integrity sha512-shrVawQFojnZv6xM40anx4CkoDP+fZsw/ZerEMsW/pyzsRbElpsL/DBVW7q3ExxwusdNXI3lXpuhEZkzs8p5Eg== + dependencies: + color-name "^1.0.0" + simple-swizzle "^0.2.2" + +color@^4.2.3: + version "4.2.3" + resolved "https://registry.npmmirror.com/color/-/color-4.2.3.tgz#d781ecb5e57224ee43ea9627560107c0e0c6463a" + integrity sha512-1rXeuUUiGGrykh+CeBdu5Ie7OJwinCgQY0bc7GCRxy5xVHy+moaqkpL/jqQq0MtQOeYcrqEz4abc5f0KtU7W4A== + dependencies: + color-convert "^2.0.1" + color-string "^1.9.0" + +colord@^2.9.1: + version "2.9.2" + resolved "https://registry.npmmirror.com/colord/-/colord-2.9.2.tgz#25e2bacbbaa65991422c07ea209e2089428effb1" + integrity sha512-Uqbg+J445nc1TKn4FoDPS6ZZqAvEDnwrH42yo8B40JSOgSLxMZ/gt3h4nmCtPLQeXhjJJkqBx7SCY35WnIixaQ== + +colorette@^1.2.2: + version "1.4.0" + resolved "https://registry.npmmirror.com/colorette/-/colorette-1.4.0.tgz#5190fbb87276259a86ad700bff2c6d6faa3fca40" + integrity sha512-Y2oEozpomLn7Q3HFP7dpww7AtMJplbM9lGZP6RDfHqmbeRjiwRg4n6VM6j4KLmRke85uWEI7JqF17f3pqdRA0g== + +combined-stream@^1.0.6, combined-stream@^1.0.8, combined-stream@~1.0.6: + version "1.0.8" + resolved "https://registry.npmmirror.com/combined-stream/-/combined-stream-1.0.8.tgz#c3d45a8b34fd730631a110a8a2520682b31d5a7f" + integrity sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg== + dependencies: + delayed-stream "~1.0.0" + +comma-separated-tokens@^1.0.0: + version "1.0.8" + resolved "https://registry.npmmirror.com/comma-separated-tokens/-/comma-separated-tokens-1.0.8.tgz#632b80b6117867a158f1080ad498b2fbe7e3f5ea" + integrity sha512-GHuDRO12Sypu2cV70d1dkA2EUmXHgntrzbpvOB+Qy+49ypNfGgFQIC2fhhXbnyrJRynDCAARsT7Ou0M6hirpfw== + +command-exists@^1.2.4: + version "1.2.9" + resolved "https://registry.npmmirror.com/command-exists/-/command-exists-1.2.9.tgz#c50725af3808c8ab0260fd60b01fbfa25b954f69" + integrity sha512-LTQ/SGc+s0Xc0Fu5WaKnR0YiygZkm9eKFvyS+fRsU7/ZWFF8ykFM6Pc9aCVf1+xasOOZpO3BAVgVrKvsqKHV7w== + +commander@^2.20.0, commander@^2.20.3: + version "2.20.3" + resolved "https://registry.npmmirror.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33" + integrity sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ== + +commander@^7.2.0: + version "7.2.0" + resolved "https://registry.npmmirror.com/commander/-/commander-7.2.0.tgz#a36cb57d0b501ce108e4d20559a150a391d97ab7" + integrity sha512-QrWXB+ZQSVPmIWIhtEO9H+gwHaMGYiF5ChvoJ+K9ZGHG/sVsa6yiesAD1GC/x46sET00Xlwo1u49RVVVzvcSkw== + +common-tags@1.8.2, common-tags@^1.8.0, common-tags@^1.8.2: + version "1.8.2" + resolved "https://registry.npmmirror.com/common-tags/-/common-tags-1.8.2.tgz#94ebb3c076d26032745fd54face7f688ef5ac9c6" + integrity sha512-gk/Z852D2Wtb//0I+kRFNKKE9dIIVirjoqPoA1wJU+XePVXZfGeBpk45+A1rKO4Q43prqWBNY/MiIeRLbPWUaA== + +commondir@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/commondir/-/commondir-1.0.1.tgz#ddd800da0c66127393cca5950ea968a3aaf1253b" + integrity sha512-W9pAhw0ja1Edb5GVdIF1mjZw/ASI0AlShXM83UUGe2DVr5TdAPEA1OA8m/g8zWp9x6On7gqufY+FatDbC3MDQg== + +component-emitter@~1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/component-emitter/-/component-emitter-1.3.0.tgz#16e4070fba8ae29b679f2215853ee181ab2eabc0" + integrity sha512-Rd3se6QB+sO1TwqZjscQrurpEPIfO0/yYnSin6Q/rD3mOutHvUrCAhJub3r90uNb+SESBuE0QYoB90YdfatsRg== + +compose-function@3.0.3: + version "3.0.3" + resolved "https://registry.npmmirror.com/compose-function/-/compose-function-3.0.3.tgz#9ed675f13cc54501d30950a486ff6a7ba3ab185f" + integrity sha512-xzhzTJ5eC+gmIzvZq+C3kCJHsp9os6tJkrigDRZclyGtOKINbZtE8n1Tzmeh32jW+BUDPbvZpibwvJHBLGMVwg== + dependencies: + arity-n "^1.0.4" + +compress-brotli@^1.3.8: + version "1.3.8" + resolved "https://registry.npmmirror.com/compress-brotli/-/compress-brotli-1.3.8.tgz#0c0a60c97a989145314ec381e84e26682e7b38db" + integrity sha512-lVcQsjhxhIXsuupfy9fmZUFtAIdBmXA7EGY6GBdgZ++qkM9zG4YFT8iU7FoBxzryNDMOpD1HIFHUSX4D87oqhQ== + dependencies: + "@types/json-buffer" "~3.0.0" + json-buffer "~3.0.1" + +compressible@~2.0.16: + version "2.0.18" + resolved "https://registry.npmmirror.com/compressible/-/compressible-2.0.18.tgz#af53cca6b070d4c3c0750fbd77286a6d7cc46fba" + integrity sha512-AF3r7P5dWxL8MxyITRMlORQNaOA2IkAFaTr4k7BUumjPtRpGDTZpl0Pb1XCO6JeDCBdp126Cgs9sMxqSjgYyRg== + dependencies: + mime-db ">= 1.43.0 < 2" + +compression@^1.7.4: + version "1.7.4" + resolved "https://registry.npmmirror.com/compression/-/compression-1.7.4.tgz#95523eff170ca57c29a0ca41e6fe131f41e5bb8f" + integrity sha512-jaSIDzP9pZVS4ZfQ+TzvtiWhdpFhE2RDHz8QJkpX9SIpLq88VueF5jJw6t+6CUQcAoA6t+x89MLrWAqpfDE8iQ== + dependencies: + accepts "~1.3.5" + bytes "3.0.0" + compressible "~2.0.16" + debug "2.6.9" + on-headers "~1.0.2" + safe-buffer "5.1.2" + vary "~1.1.2" + +compute-scroll-into-view@^1.0.17: + version "1.0.17" + resolved "https://registry.npmmirror.com/compute-scroll-into-view/-/compute-scroll-into-view-1.0.17.tgz#6a88f18acd9d42e9cf4baa6bec7e0522607ab7ab" + integrity sha512-j4dx+Fb0URmzbwwMUrhqWM2BEWHdFGx+qZ9qqASHRPqvTYdqvWnHg0H1hIbcyLnvgnoNAVMlwkepyqM3DaIFUg== + +concat-map@0.0.1: + version "0.0.1" + resolved "https://registry.npmmirror.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" + integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg== + +concat-stream@^1.5.2: + version "1.6.2" + resolved "https://registry.npmmirror.com/concat-stream/-/concat-stream-1.6.2.tgz#904bdf194cd3122fc675c77fc4ac3d4ff0fd1a34" + integrity sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw== + dependencies: + buffer-from "^1.0.0" + inherits "^2.0.3" + readable-stream "^2.2.2" + typedarray "^0.0.6" + +configstore@^5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/configstore/-/configstore-5.0.1.tgz#d365021b5df4b98cdd187d6a3b0e3f6a7cc5ed96" + integrity sha512-aMKprgk5YhBNyH25hj8wGt2+D52Sw1DRRIzqBwLp2Ya9mFmY8KPvvtvmna8SxVR9JMZ4kzMD68N22vlaRpkeFA== + dependencies: + dot-prop "^5.2.0" + graceful-fs "^4.1.2" + make-dir "^3.0.0" + unique-string "^2.0.0" + write-file-atomic "^3.0.0" + xdg-basedir "^4.0.0" + +confusing-browser-globals@^1.0.10: + version "1.0.11" + resolved "https://registry.npmmirror.com/confusing-browser-globals/-/confusing-browser-globals-1.0.11.tgz#ae40e9b57cdd3915408a2805ebd3a5585608dc81" + integrity sha512-JsPKdmh8ZkmnHxDk55FZ1TqVLvEQTvoByJZRN9jzI0UjxK/QgAmsphz7PGtqgPieQZ/CQcHWXCR7ATDNhGe+YA== + +console-control-strings@^1.0.0, console-control-strings@~1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/console-control-strings/-/console-control-strings-1.1.0.tgz#3d7cf4464db6446ea644bf4b39507f9851008e8e" + integrity sha512-ty/fTekppD2fIwRvnZAVdeOiGd1c7YXEixbgJTNzqcxJWKQnjJ/V1bNEEE6hygpM3WjwHFUVK6HTjWSzV4a8sQ== + +constant-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/constant-case/-/constant-case-3.0.4.tgz#3b84a9aeaf4cf31ec45e6bf5de91bdfb0589faf1" + integrity sha512-I2hSBi7Vvs7BEuJDr5dDHfzb/Ruj3FyvFyh7KLilAjNQw3Be+xgqUBA2W6scVEcL0hL1dwPRtIqEPVUCKkSsyQ== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + upper-case "^2.0.2" + +content-disposition@0.5.4: + version "0.5.4" + resolved "https://registry.npmmirror.com/content-disposition/-/content-disposition-0.5.4.tgz#8b82b4efac82512a02bb0b1dcec9d2c5e8eb5bfe" + integrity sha512-FveZTNuGw04cxlAiWbzi6zTAL/lhehaWbTtgluJh4/E95DqMwTmha3KZN1aAWA8cFIhHzMZUvLevkw5Rqk+tSQ== + dependencies: + safe-buffer "5.2.1" + +content-type@^1.0.4, content-type@~1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/content-type/-/content-type-1.0.4.tgz#e138cc75e040c727b1966fe5e5f8c9aee256fe3b" + integrity sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA== + +convert-hrtime@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/convert-hrtime/-/convert-hrtime-3.0.0.tgz#62c7593f5809ca10be8da858a6d2f702bcda00aa" + integrity sha512-7V+KqSvMiHp8yWDuwfww06XleMWVVB9b9tURBx+G7UTADuo5hYPuowKloz4OzOqbPezxgo+fdQ1522WzPG4OeA== + +convert-source-map@1.7.0: + version "1.7.0" + resolved "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-1.7.0.tgz#17a2cb882d7f77d3490585e2ce6c524424a3a442" + integrity sha512-4FJkXzKXEDB1snCFZlLP4gpC3JILicCpGbzG9f9G7tGqGCzETQ2hWPrcinA9oU4wtf2biUaEH5065UnMeR33oA== + dependencies: + safe-buffer "~5.1.1" + +convert-source-map@^0.3.3: + version "0.3.5" + resolved "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-0.3.5.tgz#f1d802950af7dd2631a1febe0596550c86ab3190" + integrity sha512-+4nRk0k3oEpwUB7/CalD7xE2z4VmtEnnq0GO2IPTkrooTrAhEsWvuLF5iWP1dXrwluki/azwXV1ve7gtYuPldg== + +convert-source-map@^1.7.0: + version "1.8.0" + resolved "https://registry.npmmirror.com/convert-source-map/-/convert-source-map-1.8.0.tgz#f3373c32d21b4d780dd8004514684fb791ca4369" + integrity sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA== + dependencies: + safe-buffer "~5.1.1" + +cookie-signature@1.0.6: + version "1.0.6" + resolved "https://registry.npmmirror.com/cookie-signature/-/cookie-signature-1.0.6.tgz#e303a882b342cc3ee8ca513a79999734dab3ae2c" + integrity sha512-QADzlaHc8icV8I7vbaJXJwod9HWYp8uCqf1xa4OfNu1T7JVxQIrUgOWtHdNDtPiywmFbiS12VjotIXLrKM3orQ== + +cookie@0.5.0: + version "0.5.0" + resolved "https://registry.npmmirror.com/cookie/-/cookie-0.5.0.tgz#d1f5d71adec6558c58f389987c366aa47e994f8b" + integrity sha512-YZ3GUyn/o8gfKJlnlX7g7xq4gyO6OSuhGPKaaGssGB2qgDUS0gPgtTvoyZLTt9Ab6dC4hfc9dV5arkvc/OCmrw== + +cookie@^0.4.1, cookie@~0.4.1: + version "0.4.2" + resolved "https://registry.npmmirror.com/cookie/-/cookie-0.4.2.tgz#0e41f24de5ecf317947c82fc789e06a884824432" + integrity sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA== + +copy-to-clipboard@^3.2.0: + version "3.3.1" + resolved "https://registry.npmmirror.com/copy-to-clipboard/-/copy-to-clipboard-3.3.1.tgz#115aa1a9998ffab6196f93076ad6da3b913662ae" + integrity sha512-i13qo6kIHTTpCm8/Wup+0b1mVWETvu2kIMzKoK8FpkLkFxlt0znUAHcMzox+T8sPlqtZXq3CulEjQHsYiGFJUw== + dependencies: + toggle-selection "^1.0.6" + +core-js-compat@3.9.0: + version "3.9.0" + resolved "https://registry.npmmirror.com/core-js-compat/-/core-js-compat-3.9.0.tgz#29da39385f16b71e1915565aa0385c4e0963ad56" + integrity sha512-YK6fwFjCOKWwGnjFUR3c544YsnA/7DoLL0ysncuOJ4pwbriAtOpvM2bygdlcXbvQCQZ7bBU9CL4t7tGl7ETRpQ== + dependencies: + browserslist "^4.16.3" + semver "7.0.0" + +core-js-compat@^3.21.0, core-js-compat@^3.22.1: + version "3.23.1" + resolved "https://registry.npmmirror.com/core-js-compat/-/core-js-compat-3.23.1.tgz#23d44d9f209086e60dabf9130cea7719af6e199b" + integrity sha512-KeYrEc8t6FJsKYB2qnDwRHWaC0cJNaqlHfCpMe5q3j/W1nje3moib/txNklddLPCtGb+etcBIyJ8zuMa/LN5/A== + dependencies: + browserslist "^4.20.4" + semver "7.0.0" + +core-js-pure@^3.20.2: + version "3.23.1" + resolved "https://registry.npmmirror.com/core-js-pure/-/core-js-pure-3.23.1.tgz#0b27e4c3ad46178b84e790dbbb81987218ab82ad" + integrity sha512-3qNgf6TqI3U1uhuSYRzJZGfFd4T+YlbyVPl+jgRiKjdZopvG4keZQwWZDAWpu1UH9nCgTpUzIV3GFawC7cJsqg== + +core-js@^2.4.0: + version "2.6.12" + resolved "https://registry.npmmirror.com/core-js/-/core-js-2.6.12.tgz#d9333dfa7b065e347cc5682219d6f690859cc2ec" + integrity sha512-Kb2wC0fvsWfQrgk8HU5lW6U/Lcs8+9aaYcy4ZFc6DDlo4nZ7n70dEgE5rtR0oG6ufKDUnrwfWL1mXR5ljDatrQ== + +core-js@^3.22.3: + version "3.23.1" + resolved "https://registry.npmmirror.com/core-js/-/core-js-3.23.1.tgz#9f9a9255115f62c512db56d567f636da32ca0b78" + integrity sha512-wfMYHWi1WQjpgZNC9kAlN4ut04TM9fUTdi7CqIoTVM7yaiOUQTklOzfb+oWH3r9edQcT3F887swuVmxrV+CC8w== + +core-util-is@1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/core-util-is/-/core-util-is-1.0.2.tgz#b5fd54220aa2bc5ab57aab7140c940754503c1a7" + integrity sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ== + +core-util-is@~1.0.0: + version "1.0.3" + resolved "https://registry.npmmirror.com/core-util-is/-/core-util-is-1.0.3.tgz#a6042d3634c2b27e9328f837b965fac83808db85" + integrity sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ== + +cors@^2.8.5, cors@~2.8.5: + version "2.8.5" + resolved "https://registry.npmmirror.com/cors/-/cors-2.8.5.tgz#eac11da51592dd86b9f06f6e7ac293b3df875d29" + integrity sha512-KIHbLJqu73RGr/hnbrO9uBeixNGuvSQjul/jdFvS/KFSIH1hWVd1ng7zOHx+YrEfInLG7q4n6GHQ9cDtxv/P6g== + dependencies: + object-assign "^4" + vary "^1" + +cosmiconfig-toml-loader@1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/cosmiconfig-toml-loader/-/cosmiconfig-toml-loader-1.0.0.tgz#0681383651cceff918177debe9084c0d3769509b" + integrity sha512-H/2gurFWVi7xXvCyvsWRLCMekl4tITJcX0QEsDMpzxtuxDyM59xLatYNg4s/k9AA/HdtCYfj2su8mgA0GSDLDA== + dependencies: + "@iarna/toml" "^2.2.5" + +cosmiconfig@7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/cosmiconfig/-/cosmiconfig-7.0.0.tgz#ef9b44d773959cae63ddecd122de23853b60f8d3" + integrity sha512-pondGvTuVYDk++upghXJabWzL6Kxu6f26ljFw64Swq9v6sQPUL3EUlVDV56diOjpCayKihL6hVe8exIACU4XcA== + dependencies: + "@types/parse-json" "^4.0.0" + import-fresh "^3.2.1" + parse-json "^5.0.0" + path-type "^4.0.0" + yaml "^1.10.0" + +cosmiconfig@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/cosmiconfig/-/cosmiconfig-6.0.0.tgz#da4fee853c52f6b1e6935f41c1a2fc50bd4a9982" + integrity sha512-xb3ZL6+L8b9JLLCx3ZdoZy4+2ECphCMo2PwqgP1tlfVq6M6YReyzBJtvWWtbDSpNr9hn96pkCiZqUcFEc+54Qg== + dependencies: + "@types/parse-json" "^4.0.0" + import-fresh "^3.1.0" + parse-json "^5.0.0" + path-type "^4.0.0" + yaml "^1.7.2" + +cosmiconfig@^7.0.0: + version "7.0.1" + resolved "https://registry.npmmirror.com/cosmiconfig/-/cosmiconfig-7.0.1.tgz#714d756522cace867867ccb4474c5d01bbae5d6d" + integrity sha512-a1YWNUV2HwGimB7dU2s1wUMurNKjpx60HxBB6xUM8Re+2s1g1IIfJvFR0/iCF+XHdE0GMTKTuLR32UQff4TEyQ== + dependencies: + "@types/parse-json" "^4.0.0" + import-fresh "^3.2.1" + parse-json "^5.0.0" + path-type "^4.0.0" + yaml "^1.10.0" + +create-gatsby@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/create-gatsby/-/create-gatsby-2.17.0.tgz#00d42bfca9ca548957fcd88ea4f7a297a8b02fd8" + integrity sha512-RrMAHZL7J5Ew/laz83g+UNQ2RnXszGuub3sG7DQkG0tK5mS9TqTW+FIYyzHCLhdJkr81qC/HRDFsJv0fuYISwg== + dependencies: + "@babel/runtime" "^7.15.4" + +create-require@^1.1.0: + version "1.1.1" + resolved "https://registry.npmmirror.com/create-require/-/create-require-1.1.1.tgz#c1d7e8f1e5f6cfc9ff65f9cd352d37348756c333" + integrity sha512-dcKFX3jn0MpIaXjisoRvexIJVEKzaq7z2rZKxf+MSr9TkdmHmsU4m2lcLojrj/FHl8mk5VxMmYA+ftRkP/3oKQ== + +cross-fetch@3.1.4: + version "3.1.4" + resolved "https://registry.npmmirror.com/cross-fetch/-/cross-fetch-3.1.4.tgz#9723f3a3a247bf8b89039f3a380a9244e8fa2f39" + integrity sha512-1eAtFWdIubi6T4XPy6ei9iUFoKpUkIF971QLN8lIvvvwueI65+Nw5haMNKUwfJxabqlIIDODJKGrQ66gxC0PbQ== + dependencies: + node-fetch "2.6.1" + +cross-fetch@^3.1.5: + version "3.1.5" + resolved "https://registry.npmmirror.com/cross-fetch/-/cross-fetch-3.1.5.tgz#e1389f44d9e7ba767907f7af8454787952ab534f" + integrity sha512-lvb1SBsI0Z7GDwmuid+mU3kWVBwTVUbe7S0H52yaaAdQOXq2YktTCZdlAcNKFzE6QtRz0snpw9bNiPeOIkkQvw== + dependencies: + node-fetch "2.6.7" + +cross-spawn@^6.0.0, cross-spawn@^6.0.5: + version "6.0.5" + resolved "https://registry.npmmirror.com/cross-spawn/-/cross-spawn-6.0.5.tgz#4a5ec7c64dfae22c3a14124dbacdee846d80cbc4" + integrity sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ== + dependencies: + nice-try "^1.0.4" + path-key "^2.0.1" + semver "^5.5.0" + shebang-command "^1.2.0" + which "^1.2.9" + +cross-spawn@^7.0.2, cross-spawn@^7.0.3: + version "7.0.3" + resolved "https://registry.npmmirror.com/cross-spawn/-/cross-spawn-7.0.3.tgz#f73a85b9d5d41d045551c177e2882d4ac85728a6" + integrity sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w== + dependencies: + path-key "^3.1.0" + shebang-command "^2.0.0" + which "^2.0.1" + +crypto-random-string@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/crypto-random-string/-/crypto-random-string-2.0.0.tgz#ef2a7a966ec11083388369baa02ebead229b30d5" + integrity sha512-v1plID3y9r/lPhviJ1wrXpLeyUIGAZ2SHNYTEapm7/8A9nLPoyvVp3RK/EPFqn5kEznyWgYZNsRtYYIWbuG8KA== + +css-declaration-sorter@^6.2.2: + version "6.3.0" + resolved "https://registry.npmmirror.com/css-declaration-sorter/-/css-declaration-sorter-6.3.0.tgz#72ebd995c8f4532ff0036631f7365cce9759df14" + integrity sha512-OGT677UGHJTAVMRhPO+HJ4oKln3wkBTwtDFH0ojbqm+MJm6xuDMHp2nkhh/ThaBqq20IbraBQSWKfSLNHQO9Og== + +css-loader@^5.2.7: + version "5.2.7" + resolved "https://registry.npmmirror.com/css-loader/-/css-loader-5.2.7.tgz#9b9f111edf6fb2be5dc62525644cbc9c232064ae" + integrity sha512-Q7mOvpBNBG7YrVGMxRxcBJZFL75o+cH2abNASdibkj/fffYD8qWbInZrD0S9ccI6vZclF3DsHE7njGlLtaHbhg== + dependencies: + icss-utils "^5.1.0" + loader-utils "^2.0.0" + postcss "^8.2.15" + postcss-modules-extract-imports "^3.0.0" + postcss-modules-local-by-default "^4.0.0" + postcss-modules-scope "^3.0.0" + postcss-modules-values "^4.0.0" + postcss-value-parser "^4.1.0" + schema-utils "^3.0.0" + semver "^7.3.5" + +css-minimizer-webpack-plugin@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/css-minimizer-webpack-plugin/-/css-minimizer-webpack-plugin-2.0.0.tgz#3c42f6624ed4cf4780dd963e23ee649e5a25c1a8" + integrity sha512-cG/uc94727tx5pBNtb1Sd7gvUPzwmcQi1lkpfqTpdkuNq75hJCw7bIVsCNijLm4dhDcr1atvuysl2rZqOG8Txw== + dependencies: + cssnano "^5.0.0" + jest-worker "^26.3.0" + p-limit "^3.0.2" + postcss "^8.2.9" + schema-utils "^3.0.0" + serialize-javascript "^5.0.1" + source-map "^0.6.1" + +css-select-base-adapter@^0.1.1: + version "0.1.1" + resolved "https://registry.npmmirror.com/css-select-base-adapter/-/css-select-base-adapter-0.1.1.tgz#3b2ff4972cc362ab88561507a95408a1432135d7" + integrity sha512-jQVeeRG70QI08vSTwf1jHxp74JoZsr2XSgETae8/xC8ovSnL2WF87GTLO86Sbwdt2lK4Umg4HnnwMO4YF3Ce7w== + +css-select@^2.0.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/css-select/-/css-select-2.1.0.tgz#6a34653356635934a81baca68d0255432105dbef" + integrity sha512-Dqk7LQKpwLoH3VovzZnkzegqNSuAziQyNZUcrdDM401iY+R5NkGBXGmtO05/yaXQziALuPogeG0b7UAgjnTJTQ== + dependencies: + boolbase "^1.0.0" + css-what "^3.2.1" + domutils "^1.7.0" + nth-check "^1.0.2" + +css-select@^4.1.3: + version "4.3.0" + resolved "https://registry.npmmirror.com/css-select/-/css-select-4.3.0.tgz#db7129b2846662fd8628cfc496abb2b59e41529b" + integrity sha512-wPpOYtnsVontu2mODhA19JrqWxNsfdatRKd64kmpRbQgh1KtItko5sTnEpPdpSaJszTOhEMlF/RPz28qj4HqhQ== + dependencies: + boolbase "^1.0.0" + css-what "^6.0.1" + domhandler "^4.3.1" + domutils "^2.8.0" + nth-check "^2.0.1" + +css-select@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/css-select/-/css-select-5.1.0.tgz#b8ebd6554c3637ccc76688804ad3f6a6fdaea8a6" + integrity sha512-nwoRF1rvRRnnCqqY7updORDsuqKzqYJ28+oSMaJMMgOauh3fvwHqMS7EZpIPqK8GL+g9mKxF1vP/ZjSeNjEVHg== + dependencies: + boolbase "^1.0.0" + css-what "^6.1.0" + domhandler "^5.0.2" + domutils "^3.0.1" + nth-check "^2.0.1" + +css-selector-parser@^1.0.0: + version "1.4.1" + resolved "https://registry.npmmirror.com/css-selector-parser/-/css-selector-parser-1.4.1.tgz#03f9cb8a81c3e5ab2c51684557d5aaf6d2569759" + integrity sha512-HYPSb7y/Z7BNDCOrakL4raGO2zltZkbeXyAd6Tg9obzix6QhzxCotdBl6VT0Dv4vZfJGVz3WL/xaEI9Ly3ul0g== + +css-tree@1.0.0-alpha.37: + version "1.0.0-alpha.37" + resolved "https://registry.npmmirror.com/css-tree/-/css-tree-1.0.0-alpha.37.tgz#98bebd62c4c1d9f960ec340cf9f7522e30709a22" + integrity sha512-DMxWJg0rnz7UgxKT0Q1HU/L9BeJI0M6ksor0OgqOnF+aRCDWg/N2641HmVyU9KVIu0OVVWOb2IpC9A+BJRnejg== + dependencies: + mdn-data "2.0.4" + source-map "^0.6.1" + +css-tree@^1.1.2, css-tree@^1.1.3: + version "1.1.3" + resolved "https://registry.npmmirror.com/css-tree/-/css-tree-1.1.3.tgz#eb4870fb6fd7707327ec95c2ff2ab09b5e8db91d" + integrity sha512-tRpdppF7TRazZrjJ6v3stzv93qxRcSsFmW6cX0Zm2NVKpxE1WV1HblnghVv9TreireHkqI/VDEsfolRF1p6y7Q== + dependencies: + mdn-data "2.0.14" + source-map "^0.6.1" + +css-what@^3.2.1: + version "3.4.2" + resolved "https://registry.npmmirror.com/css-what/-/css-what-3.4.2.tgz#ea7026fcb01777edbde52124e21f327e7ae950e4" + integrity sha512-ACUm3L0/jiZTqfzRM3Hi9Q8eZqd6IK37mMWPLz9PJxkLWllYeRf+EHUSHYEtFop2Eqytaq1FizFVh7XfBnXCDQ== + +css-what@^6.0.1, css-what@^6.1.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/css-what/-/css-what-6.1.0.tgz#fb5effcf76f1ddea2c81bdfaa4de44e79bac70f4" + integrity sha512-HTUrgRJ7r4dsZKU6GjmpfRK1O76h97Z8MfS1G0FozR+oF2kG6Vfe8JE6zwrkbxigziPHinCJ+gCPjA9EaBDtRw== + +css.escape@^1.5.1: + version "1.5.1" + resolved "https://registry.npmmirror.com/css.escape/-/css.escape-1.5.1.tgz#42e27d4fa04ae32f931a4b4d4191fa9cddee97cb" + integrity sha512-YUifsXXuknHlUsmlgyY0PKzgPOr7/FjCePfHNt0jxm83wHZi44VDMQ7/fGNkjY3/jV1MC+1CmZbaHzugyeRtpg== + +css@^2.0.0: + version "2.2.4" + resolved "https://registry.npmmirror.com/css/-/css-2.2.4.tgz#c646755c73971f2bba6a601e2cf2fd71b1298929" + integrity sha512-oUnjmWpy0niI3x/mPL8dVEI1l7MnG3+HHyRPHf+YFSbK+svOhXpmSOcDURUh2aOCgl2grzrOPt1nHLuCVFULLw== + dependencies: + inherits "^2.0.3" + source-map "^0.6.1" + source-map-resolve "^0.5.2" + urix "^0.1.0" + +cssesc@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/cssesc/-/cssesc-3.0.0.tgz#37741919903b868565e1c09ea747445cd18983ee" + integrity sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg== + +cssfilter@0.0.10: + version "0.0.10" + resolved "https://registry.npmmirror.com/cssfilter/-/cssfilter-0.0.10.tgz#c6d2672632a2e5c83e013e6864a42ce8defd20ae" + integrity sha512-FAaLDaplstoRsDR8XGYH51znUN0UY7nMc6Z9/fvE8EXGwvJE9hu7W2vHwx1+bd6gCYnln9nLbzxFTrcO9YQDZw== + +cssnano-preset-default@^5.2.11: + version "5.2.11" + resolved "https://registry.npmmirror.com/cssnano-preset-default/-/cssnano-preset-default-5.2.11.tgz#28350471bc1af9df14052472b61340347f453a53" + integrity sha512-4PadR1NtuaIK8MvLNuY7MznK4WJteldGlzCiMaaTiOUP+apeiIvUDIXykzUOoqgOOUAHrU64ncdD90NfZR3LSQ== + dependencies: + css-declaration-sorter "^6.2.2" + cssnano-utils "^3.1.0" + postcss-calc "^8.2.3" + postcss-colormin "^5.3.0" + postcss-convert-values "^5.1.2" + postcss-discard-comments "^5.1.2" + postcss-discard-duplicates "^5.1.0" + postcss-discard-empty "^5.1.1" + postcss-discard-overridden "^5.1.0" + postcss-merge-longhand "^5.1.5" + postcss-merge-rules "^5.1.2" + postcss-minify-font-values "^5.1.0" + postcss-minify-gradients "^5.1.1" + postcss-minify-params "^5.1.3" + postcss-minify-selectors "^5.2.1" + postcss-normalize-charset "^5.1.0" + postcss-normalize-display-values "^5.1.0" + postcss-normalize-positions "^5.1.0" + postcss-normalize-repeat-style "^5.1.0" + postcss-normalize-string "^5.1.0" + postcss-normalize-timing-functions "^5.1.0" + postcss-normalize-unicode "^5.1.0" + postcss-normalize-url "^5.1.0" + postcss-normalize-whitespace "^5.1.1" + postcss-ordered-values "^5.1.2" + postcss-reduce-initial "^5.1.0" + postcss-reduce-transforms "^5.1.0" + postcss-svgo "^5.1.0" + postcss-unique-selectors "^5.1.1" + +cssnano-utils@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/cssnano-utils/-/cssnano-utils-3.1.0.tgz#95684d08c91511edfc70d2636338ca37ef3a6861" + integrity sha512-JQNR19/YZhz4psLX/rQ9M83e3z2Wf/HdJbryzte4a3NSuafyp9w/I4U+hx5C2S9g41qlstH7DEWnZaaj83OuEA== + +cssnano@^5.0.0: + version "5.1.11" + resolved "https://registry.npmmirror.com/cssnano/-/cssnano-5.1.11.tgz#3bb003380718c7948ce3813493370e8946caf04b" + integrity sha512-2nx+O6LvewPo5EBtYrKc8762mMkZRk9cMGIOP4UlkmxHm7ObxH+zvsJJ+qLwPkUc4/yumL/qJkavYi9NlodWIQ== + dependencies: + cssnano-preset-default "^5.2.11" + lilconfig "^2.0.3" + yaml "^1.10.2" + +csso@^4.0.2, csso@^4.2.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/csso/-/csso-4.2.0.tgz#ea3a561346e8dc9f546d6febedd50187cf389529" + integrity sha512-wvlcdIbf6pwKEk7vHj8/Bkc0B4ylXZruLvOgs9doS5eOsOpuodOV2zJChSpkp+pRpYQLQMeF04nr3Z68Sta9jA== + dependencies: + css-tree "^1.1.2" + +csstype@3.0.9: + version "3.0.9" + resolved "https://registry.npmmirror.com/csstype/-/csstype-3.0.9.tgz#6410af31b26bd0520933d02cbc64fce9ce3fbf0b" + integrity sha512-rpw6JPxK6Rfg1zLOYCSwle2GFOOsnjmDYDaBwEcwoOg4qlsIVCN789VkBZDJAGi4T07gI4YSutR43t9Zz4Lzuw== + +csstype@^3.0.2: + version "3.1.0" + resolved "https://registry.npmmirror.com/csstype/-/csstype-3.1.0.tgz#4ddcac3718d787cf9df0d1b7d15033925c8f29f2" + integrity sha512-uX1KG+x9h5hIJsaKR9xHUeUraxf8IODOwq9JLNPq6BwB04a/xgpq3rcx47l5BZu5zBPlgD342tdke3Hom/nJRA== + +d@1, d@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/d/-/d-1.0.1.tgz#8698095372d58dbee346ffd0c7093f99f8f9eb5a" + integrity sha512-m62ShEObQ39CfralilEQRjH6oAMtNCV1xJyEx5LpRYUVN+EviphDgUc/F3hnYbADmkiNs67Y+3ylmlG7Lnu+FA== + dependencies: + es5-ext "^0.10.50" + type "^1.0.1" + +damerau-levenshtein@^1.0.7: + version "1.0.8" + resolved "https://registry.npmmirror.com/damerau-levenshtein/-/damerau-levenshtein-1.0.8.tgz#b43d286ccbd36bc5b2f7ed41caf2d0aba1f8a6e7" + integrity sha512-sdQSFB7+llfUcQHUQO3+B8ERRj0Oa4w9POWMI/puGtuf7gFywGmkaLCElnudfTiKZV+NvHqL0ifzdrI8Ro7ESA== + +dashdash@^1.12.0: + version "1.14.1" + resolved "https://registry.npmmirror.com/dashdash/-/dashdash-1.14.1.tgz#853cfa0f7cbe2fed5de20326b8dd581035f6e2f0" + integrity sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g== + dependencies: + assert-plus "^1.0.0" + +dataloader@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/dataloader/-/dataloader-2.0.0.tgz#41eaf123db115987e21ca93c005cd7753c55fe6f" + integrity sha512-YzhyDAwA4TaQIhM5go+vCLmU0UikghC/t9DTQYZR2M/UvZ1MdOhPezSDZcjj9uqQJOMqjLcpWtyW2iNINdlatQ== + +date-fns@2.x, date-fns@^2.25.0: + version "2.28.0" + resolved "https://registry.npmmirror.com/date-fns/-/date-fns-2.28.0.tgz#9570d656f5fc13143e50c975a3b6bbeb46cd08b2" + integrity sha512-8d35hViGYx/QH0icHYCeLmsLmMUheMmTyV9Fcm6gvNwdw31yXXH+O85sOBJ+OLnLQMKZowvpKb6FgMIQjcpvQw== + +dayjs@1.x: + version "1.11.3" + resolved "https://registry.npmmirror.com/dayjs/-/dayjs-1.11.3.tgz#4754eb694a624057b9ad2224b67b15d552589258" + integrity sha512-xxwlswWOlGhzgQ4TKzASQkUhqERI3egRNqgV4ScR8wlANA/A9tZ7miXa44vTTKEq5l7vWoL5G57bG3zA+Kow0A== + +debug@2, debug@2.6.9, debug@^2.6.0, debug@^2.6.9: + version "2.6.9" + resolved "https://registry.npmmirror.com/debug/-/debug-2.6.9.tgz#5d128515df134ff327e90a4c93f4e077a536341f" + integrity sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA== + dependencies: + ms "2.0.0" + +debug@^3.0.0, debug@^3.0.1, debug@^3.1.0, debug@^3.2.6, debug@^3.2.7: + version "3.2.7" + resolved "https://registry.npmmirror.com/debug/-/debug-3.2.7.tgz#72580b7e9145fb39b6676f9c5e5fb100b934179a" + integrity sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ== + dependencies: + ms "^2.1.1" + +debug@^4.0.0, debug@^4.0.1, debug@^4.1.0, debug@^4.1.1, debug@^4.3.1, debug@^4.3.4, debug@~4.3.1: + version "4.3.4" + resolved "https://registry.npmmirror.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" + integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== + dependencies: + ms "2.1.2" + +decamelize-keys@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/decamelize-keys/-/decamelize-keys-1.1.0.tgz#d171a87933252807eb3cb61dc1c1445d078df2d9" + integrity sha512-ocLWuYzRPoS9bfiSdDd3cxvrzovVMZnRDVEzAs+hWIVXGDbHxWMECij2OBuyB/An0FFW/nLuq6Kv1i/YC5Qfzg== + dependencies: + decamelize "^1.1.0" + map-obj "^1.0.0" + +decamelize@^1.1.0, decamelize@^1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290" + integrity sha512-z2S+W9X73hAUUki+N+9Za2lBlun89zigOyGrsax+KUQ6wKW4ZoWpEYBkGhQjwAjjDCkWxhY0VKEhk8wzY7F5cA== + +decode-uri-component@^0.2.0: + version "0.2.0" + resolved "https://registry.npmmirror.com/decode-uri-component/-/decode-uri-component-0.2.0.tgz#eb3913333458775cb84cd1a1fae062106bb87545" + integrity sha512-hjf+xovcEn31w/EUYdTXQh/8smFL/dzYjohQGEIgjyNavaJfBY2p5F527Bo1VPATxv0VYTUC2bOcXvqFwk78Og== + +decompress-response@^3.3.0: + version "3.3.0" + resolved "https://registry.npmmirror.com/decompress-response/-/decompress-response-3.3.0.tgz#80a4dd323748384bfa248083622aedec982adff3" + integrity sha512-BzRPQuY1ip+qDonAOz42gRm/pg9F768C+npV/4JOsxRC2sq+Rlk+Q4ZCAsOhnIaMrgarILY+RMUIvMmmX1qAEA== + dependencies: + mimic-response "^1.0.0" + +decompress-response@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/decompress-response/-/decompress-response-6.0.0.tgz#ca387612ddb7e104bd16d85aab00d5ecf09c66fc" + integrity sha512-aW35yZM6Bb/4oJlZncMH2LCoZtJXTRxES17vE3hoRiowU2kWHaJKFkSBDnDR+cm9J+9QhXmREyIfv0pji9ejCQ== + dependencies: + mimic-response "^3.1.0" + +deep-extend@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/deep-extend/-/deep-extend-0.6.0.tgz#c4fa7c95404a17a9c3e8ca7e1537312b736330ac" + integrity sha512-LOHxIOaPYdHlJRtCQfDIVZtfw/ufM8+rVj649RIHzcm/vGwQRXFt6OPqIFWsm2XEMrNIEtWR64sY1LEKD2vAOA== + +deep-is@^0.1.3: + version "0.1.4" + resolved "https://registry.npmmirror.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" + integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== + +deepmerge@^4.0, deepmerge@^4.2.2: + version "4.2.2" + resolved "https://registry.npmmirror.com/deepmerge/-/deepmerge-4.2.2.tgz#44d2ea3679b8f4d4ffba33f03d865fc1e7bf4955" + integrity sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg== + +defer-to-connect@^1.0.1: + version "1.1.3" + resolved "https://registry.npmmirror.com/defer-to-connect/-/defer-to-connect-1.1.3.tgz#331ae050c08dcf789f8c83a7b81f0ed94f4ac591" + integrity sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ== + +defer-to-connect@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/defer-to-connect/-/defer-to-connect-2.0.1.tgz#8016bdb4143e4632b77a3449c6236277de520587" + integrity sha512-4tvttepXG1VaYGrRibk5EwJd1t4udunSOVMdLSAL6mId1ix438oPwPZMALY41FCijukO1L0twNcGsdzS7dHgDg== + +define-lazy-prop@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz#3f7ae421129bcaaac9bc74905c98a0009ec9ee7f" + integrity sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og== + +define-properties@^1.1.3, define-properties@^1.1.4: + version "1.1.4" + resolved "https://registry.npmmirror.com/define-properties/-/define-properties-1.1.4.tgz#0b14d7bd7fbeb2f3572c3a7eda80ea5d57fb05b1" + integrity sha512-uckOqKcfaVvtBdsVkdPv3XjveQJsNQqmhXgRi8uhvWWuPYZCNlzT8qAyblUgNoXdHdjMTzAqeGjAoli8f+bzPA== + dependencies: + has-property-descriptors "^1.0.0" + object-keys "^1.1.1" + +defined@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/defined/-/defined-1.0.0.tgz#c98d9bcef75674188e110969151199e39b1fa693" + integrity sha512-Y2caI5+ZwS5c3RiNDJ6u53VhQHv+hHKwhkI1iHvceKUHw9Df6EK2zRLfjejRgMuCuxK7PfSWIMwWecceVvThjQ== + +delayed-stream@~1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/delayed-stream/-/delayed-stream-1.0.0.tgz#df3ae199acadfb7d440aaae0b29e2272b24ec619" + integrity sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ== + +delegates@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/delegates/-/delegates-1.0.0.tgz#84c6e159b81904fdca59a0ef44cd870d31250f9a" + integrity sha512-bd2L678uiWATM6m5Z1VzNCErI3jiGzt6HGY8OVICs40JQq/HALfbyNJmp0UDakEY4pMMaN0Ly5om/B1VI/+xfQ== + +depd@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/depd/-/depd-2.0.0.tgz#b696163cc757560d09cf22cc8fad1571b79e76df" + integrity sha512-g7nH6P6dyDioJogAAGprGpCtVImJhpPk/roCzdb3fIh61/s/nPsfR6onyMwkCAR/OlC3yBC0lESvUoQEAssIrw== + +depd@~1.1.2: + version "1.1.2" + resolved "https://registry.npmmirror.com/depd/-/depd-1.1.2.tgz#9bcd52e14c097763e749b274c4346ed2e560b5a9" + integrity sha512-7emPTl6Dpo6JRXOXjLRxck+FlLRX5847cLKEn00PLAgc3g2hTZZgr+e4c2v6QpSmLeFP3n5yUo7ft6avBK/5jQ== + +dependency-graph@^0.11.0: + version "0.11.0" + resolved "https://registry.npmmirror.com/dependency-graph/-/dependency-graph-0.11.0.tgz#ac0ce7ed68a54da22165a85e97a01d53f5eb2e27" + integrity sha512-JeMq7fEshyepOWDfcfHK06N3MhyPhz++vtqWhMT5O9A3K42rdsEDpfdVqjaqaAhsw6a+ZqeDvQVtD0hFHQWrzg== + +destroy@1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/destroy/-/destroy-1.2.0.tgz#4803735509ad8be552934c67df614f94e66fa015" + integrity sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg== + +detect-libc@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/detect-libc/-/detect-libc-1.0.3.tgz#fa137c4bd698edf55cd5cd02ac559f91a4c4ba9b" + integrity sha512-pGjwhsmsp4kL2RTz08wcOlGN83otlqHeD/Z5T8GXZB+/YcpQ/dgo+lbU8ZsGxV0HIvqqxo9l7mqYwyYMD9bKDg== + +detect-libc@^2.0.0, detect-libc@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/detect-libc/-/detect-libc-2.0.1.tgz#e1897aa88fa6ad197862937fbc0441ef352ee0cd" + integrity sha512-463v3ZeIrcWtdgIg6vI6XUncguvr2TnGl4SzDXinkt9mSLpBJKXT3mW6xT3VQdDN11+WVs29pgvivTc4Lp8v+w== + +detect-port-alt@^1.1.6: + version "1.1.6" + resolved "https://registry.npmmirror.com/detect-port-alt/-/detect-port-alt-1.1.6.tgz#24707deabe932d4a3cf621302027c2b266568275" + integrity sha512-5tQykt+LqfJFBEYaDITx7S7cR7mJ/zQmLXZ2qt5w04ainYZw6tBf9dBunMjVeVOdYVRUzUOE4HkY5J7+uttb5Q== + dependencies: + address "^1.0.1" + debug "^2.6.0" + +detect-port@^1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/detect-port/-/detect-port-1.3.0.tgz#d9c40e9accadd4df5cac6a782aefd014d573d1f1" + integrity sha512-E+B1gzkl2gqxt1IhUzwjrxBKRqx1UzC3WLONHinn8S3T6lwV/agVCyitiFOsGJ/eYuEUBvD71MZHy3Pv1G9doQ== + dependencies: + address "^1.0.1" + debug "^2.6.0" + +detective@^5.2.1: + version "5.2.1" + resolved "https://registry.npmmirror.com/detective/-/detective-5.2.1.tgz#6af01eeda11015acb0e73f933242b70f24f91034" + integrity sha512-v9XE1zRnz1wRtgurGu0Bs8uHKFSTdteYZNbIPFVhUZ39L/S79ppMpdmVOZAnoz1jfEFodc48n6MX483Xo3t1yw== + dependencies: + acorn-node "^1.8.2" + defined "^1.0.0" + minimist "^1.2.6" + +devcert@^1.2.0: + version "1.2.2" + resolved "https://registry.npmmirror.com/devcert/-/devcert-1.2.2.tgz#7ebbefeb74991e269ceafcd66f2bb78677c12652" + integrity sha512-UsLqvtJGPiGwsIZnJINUnFYaWgK7CroreGRndWHZkRD58tPFr3pVbbSyHR8lbh41+azR4jKvuNZ+eCoBZGA5kA== + dependencies: + "@types/configstore" "^2.1.1" + "@types/debug" "^0.0.30" + "@types/get-port" "^3.2.0" + "@types/glob" "^5.0.34" + "@types/lodash" "^4.14.92" + "@types/mkdirp" "^0.5.2" + "@types/node" "^8.5.7" + "@types/rimraf" "^2.0.2" + "@types/tmp" "^0.0.33" + application-config-path "^0.1.0" + command-exists "^1.2.4" + debug "^3.1.0" + eol "^0.9.1" + get-port "^3.2.0" + glob "^7.1.2" + is-valid-domain "^0.1.6" + lodash "^4.17.4" + mkdirp "^0.5.1" + password-prompt "^1.0.4" + rimraf "^2.6.2" + sudo-prompt "^8.2.0" + tmp "^0.0.33" + tslib "^1.10.0" + +dicer@0.2.5: + version "0.2.5" + resolved "https://registry.npmmirror.com/dicer/-/dicer-0.2.5.tgz#5996c086bb33218c812c090bddc09cd12facb70f" + integrity sha512-FDvbtnq7dzlPz0wyYlOExifDEZcu8h+rErEXgfxqmLfRfC/kJidEFh4+effJRO3P0xmfqyPbSMG0LveNRfTKVg== + dependencies: + readable-stream "1.1.x" + streamsearch "0.1.2" + +didyoumean@^1.2.2: + version "1.2.2" + resolved "https://registry.npmmirror.com/didyoumean/-/didyoumean-1.2.2.tgz#989346ffe9e839b4555ecf5666edea0d3e8ad037" + integrity sha512-gxtyfqMg7GKyhQmb056K7M3xszy/myH8w+B4RT+QXBQsvAOdc3XymqDDPHx1BgPgsdAA5SIifona89YtRATDzw== + +diff@^4.0.1: + version "4.0.2" + resolved "https://registry.npmmirror.com/diff/-/diff-4.0.2.tgz#60f3aecb89d5fae520c11aa19efc2bb982aade7d" + integrity sha512-58lmxKSA4BNyLz+HHMUzlOEpg09FV+ev6ZMe3vJihgdxzgcwZ8VoEEPmALCZG9LmqfVoNMMKpttIYTVG6uDY7A== + +dir-glob@^3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/dir-glob/-/dir-glob-3.0.1.tgz#56dbf73d992a4a93ba1584f4534063fd2e41717f" + integrity sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA== + dependencies: + path-type "^4.0.0" + +dlv@^1.1.3: + version "1.1.3" + resolved "https://registry.npmmirror.com/dlv/-/dlv-1.1.3.tgz#5c198a8a11453596e751494d49874bc7732f2e79" + integrity sha512-+HlytyjlPKnIG8XuRG8WvmBP8xs8P71y+SKKS6ZXWoEgLuePxtDoUEiH7WkdePWrQ5JBpE6aoVqfZfJUQkjXwA== + +doctrine@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/doctrine/-/doctrine-2.1.0.tgz#5cd01fc101621b42c4cd7f5d1a66243716d3f39d" + integrity sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw== + dependencies: + esutils "^2.0.2" + +doctrine@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/doctrine/-/doctrine-3.0.0.tgz#addebead72a6574db783639dc87a121773973961" + integrity sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w== + dependencies: + esutils "^2.0.2" + +dom-align@^1.7.0: + version "1.12.3" + resolved "https://registry.npmmirror.com/dom-align/-/dom-align-1.12.3.tgz#a36d02531dae0eefa2abb0c4db6595250526f103" + integrity sha512-Gj9hZN3a07cbR6zviMUBOMPdWxYhbMI+x+WS0NAIu2zFZmbK8ys9R79g+iG9qLnlCwpFoaB+fKy8Pdv470GsPA== + +dom-converter@^0.2.0: + version "0.2.0" + resolved "https://registry.npmmirror.com/dom-converter/-/dom-converter-0.2.0.tgz#6721a9daee2e293682955b6afe416771627bb768" + integrity sha512-gd3ypIPfOMr9h5jIKq8E3sHOTCjeirnl0WK5ZdS1AW0Odt0b1PaWaHdJ4Qk4klv+YB9aJBS7mESXjFoDQPu6DA== + dependencies: + utila "~0.4" + +dom-serializer@0: + version "0.2.2" + resolved "https://registry.npmmirror.com/dom-serializer/-/dom-serializer-0.2.2.tgz#1afb81f533717175d478655debc5e332d9f9bb51" + integrity sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g== + dependencies: + domelementtype "^2.0.1" + entities "^2.0.0" + +dom-serializer@^1.0.1: + version "1.4.1" + resolved "https://registry.npmmirror.com/dom-serializer/-/dom-serializer-1.4.1.tgz#de5d41b1aea290215dc45a6dae8adcf1d32e2d30" + integrity sha512-VHwB3KfrcOOkelEG2ZOfxqLZdfkil8PtJi4P8N2MMXucZq2yLp75ClViUlOVwyoHEDjYU433Aq+5zWP61+RGag== + dependencies: + domelementtype "^2.0.1" + domhandler "^4.2.0" + entities "^2.0.0" + +dom-serializer@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/dom-serializer/-/dom-serializer-2.0.0.tgz#e41b802e1eedf9f6cae183ce5e622d789d7d8e53" + integrity sha512-wIkAryiqt/nV5EQKqQpo3SToSOV9J0DnbJqwK7Wv/Trc92zIAYZ4FlMu+JPFW1DfGFt81ZTCGgDEabffXeLyJg== + dependencies: + domelementtype "^2.3.0" + domhandler "^5.0.2" + entities "^4.2.0" + +dom-walk@^0.1.0: + version "0.1.2" + resolved "https://registry.npmmirror.com/dom-walk/-/dom-walk-0.1.2.tgz#0c548bef048f4d1f2a97249002236060daa3fd84" + integrity sha512-6QvTW9mrGeIegrFXdtQi9pk7O/nSK6lSdXW2eqUspN5LWD7UTji2Fqw5V2YLjBpHEoU9Xl/eUWNpDeZvoyOv2w== + +domelementtype@1: + version "1.3.1" + resolved "https://registry.npmmirror.com/domelementtype/-/domelementtype-1.3.1.tgz#d048c44b37b0d10a7f2a3d5fee3f4333d790481f" + integrity sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w== + +domelementtype@^2.0.1, domelementtype@^2.2.0, domelementtype@^2.3.0: + version "2.3.0" + resolved "https://registry.npmmirror.com/domelementtype/-/domelementtype-2.3.0.tgz#5c45e8e869952626331d7aab326d01daf65d589d" + integrity sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw== + +domhandler@4.2.2: + version "4.2.2" + resolved "https://registry.npmmirror.com/domhandler/-/domhandler-4.2.2.tgz#e825d721d19a86b8c201a35264e226c678ee755f" + integrity sha512-PzE9aBMsdZO8TK4BnuJwH0QT41wgMbRzuZrHUcpYncEjmQazq8QEaBWgLG7ZyC/DAZKEgglpIA6j4Qn/HmxS3w== + dependencies: + domelementtype "^2.2.0" + +domhandler@^3.0.0: + version "3.3.0" + resolved "https://registry.npmmirror.com/domhandler/-/domhandler-3.3.0.tgz#6db7ea46e4617eb15cf875df68b2b8524ce0037a" + integrity sha512-J1C5rIANUbuYK+FuFL98650rihynUOEzRLxW+90bKZRWB6A1X1Tf82GxR1qAWLyfNPRvjqfip3Q5tdYlmAa9lA== + dependencies: + domelementtype "^2.0.1" + +domhandler@^4.0.0, domhandler@^4.2.0, domhandler@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/domhandler/-/domhandler-4.3.1.tgz#8d792033416f59d68bc03a5aa7b018c1ca89279c" + integrity sha512-GrwoxYN+uWlzO8uhUXRl0P+kHE4GtVPfYzVLcUxPL7KNdHKj66vvlhiweIHqYYXWlw+T8iLMp42Lm67ghw4WMQ== + dependencies: + domelementtype "^2.2.0" + +domhandler@^5.0.1, domhandler@^5.0.2, domhandler@^5.0.3: + version "5.0.3" + resolved "https://registry.npmmirror.com/domhandler/-/domhandler-5.0.3.tgz#cc385f7f751f1d1fc650c21374804254538c7d31" + integrity sha512-cgwlv/1iFQiFnU96XXgROh8xTeetsnJiDsTc7TYCLFd9+/WNkIqPTxiM/8pSd8VIrhXGTf1Ny1q1hquVqDJB5w== + dependencies: + domelementtype "^2.3.0" + +domutils@^1.7.0: + version "1.7.0" + resolved "https://registry.npmmirror.com/domutils/-/domutils-1.7.0.tgz#56ea341e834e06e6748af7a1cb25da67ea9f8c2a" + integrity sha512-Lgd2XcJ/NjEw+7tFvfKxOzCYKZsdct5lczQ2ZaQY8Djz7pfAD3Gbp8ySJWtreII/vDlMVmxwa6pHmdxIYgttDg== + dependencies: + dom-serializer "0" + domelementtype "1" + +domutils@^2.0.0, domutils@^2.5.2, domutils@^2.8.0: + version "2.8.0" + resolved "https://registry.npmmirror.com/domutils/-/domutils-2.8.0.tgz#4437def5db6e2d1f5d6ee859bd95ca7d02048135" + integrity sha512-w96Cjofp72M5IIhpjgobBimYEfoPjx1Vx0BSX9P30WBdZW2WIKU0T1Bd0kz2eNZ9ikjKgHbEyKx8BB6H1L3h3A== + dependencies: + dom-serializer "^1.0.1" + domelementtype "^2.2.0" + domhandler "^4.2.0" + +domutils@^3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/domutils/-/domutils-3.0.1.tgz#696b3875238338cb186b6c0612bd4901c89a4f1c" + integrity sha512-z08c1l761iKhDFtfXO04C7kTdPBLi41zwOZl00WS8b5eiaebNpY00HKbztwBq+e3vyqWNwWF3mP9YLUeqIrF+Q== + dependencies: + dom-serializer "^2.0.0" + domelementtype "^2.3.0" + domhandler "^5.0.1" + +dot-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/dot-case/-/dot-case-3.0.4.tgz#9b2b670d00a431667a8a75ba29cd1b98809ce751" + integrity sha512-Kv5nKlh6yRrdrGvxeJ2e5y2eRUpkUosIW4A2AS38zwSz27zu7ufDwQPi5Jhs3XAlGNetl3bmnGhQsMtkKJnj3w== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + +dot-prop@^5.2.0: + version "5.3.0" + resolved "https://registry.npmmirror.com/dot-prop/-/dot-prop-5.3.0.tgz#90ccce708cd9cd82cc4dc8c3ddd9abdd55b20e88" + integrity sha512-QM8q3zDe58hqUqjraQOmzZ1LIH9SWQJTlEKCH4kJ2oQvLZk7RbQXvtDM2XEq3fwkV9CCvvH4LA0AV+ogFsBM2Q== + dependencies: + is-obj "^2.0.0" + +dotenv-expand@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/dotenv-expand/-/dotenv-expand-5.1.0.tgz#3fbaf020bfd794884072ea26b1e9791d45a629f0" + integrity sha512-YXQl1DSa4/PQyRfgrv6aoNjhasp/p4qs9FjJ4q4cQk+8m4r6k4ZSiEyytKG8f8W9gi8WsQtIObNmKd+tMzNTmA== + +dotenv@^7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/dotenv/-/dotenv-7.0.0.tgz#a2be3cd52736673206e8a85fb5210eea29628e7c" + integrity sha512-M3NhsLbV1i6HuGzBUH8vXrtxOk+tWmzWKDMbAVSUp3Zsjm7ywFeuwrUXhmhQyRK1q5B5GGy7hcXPbj3bnfZg2g== + +dotenv@^8.6.0: + version "8.6.0" + resolved "https://registry.npmmirror.com/dotenv/-/dotenv-8.6.0.tgz#061af664d19f7f4d8fc6e4ff9b584ce237adcb8b" + integrity sha512-IrPdXQsk2BbzvCBGBOTmmSH5SodmqZNt4ERAZDmW4CT+tL8VtvinqywuANaFu4bOMWki16nqf0e4oC0QIaDr/g== + +duplexer2@^0.1.2: + version "0.1.4" + resolved "https://registry.npmmirror.com/duplexer2/-/duplexer2-0.1.4.tgz#8b12dab878c0d69e3e7891051662a32fc6bddcc1" + integrity sha512-asLFVfWWtJ90ZyOUHMqk7/S2w2guQKxUI2itj3d92ADHhxUSbCMGi1f1cBcJ7xM1To+pE/Khbwo1yuNbMEPKeA== + dependencies: + readable-stream "^2.0.2" + +duplexer3@^0.1.4: + version "0.1.4" + resolved "https://registry.npmmirror.com/duplexer3/-/duplexer3-0.1.4.tgz#ee01dd1cac0ed3cbc7fdbea37dc0a8f1ce002ce2" + integrity sha512-CEj8FwwNA4cVH2uFCoHUrmojhYh1vmCdOaneKJXwkeY1i9jnlslVo9dx+hQ5Hl9GnH/Bwy/IjxAyOePyPKYnzA== + +duplexer@^0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/duplexer/-/duplexer-0.1.2.tgz#3abe43aef3835f8ae077d136ddce0f276b0400e6" + integrity sha512-jtD6YG370ZCIi/9GTaJKQxWTZD045+4R4hTk/x1UyoqadyJ9x9CgSi1RlVDQF8U2sxLLSnFkCaMihqljHIWgMg== + +ecc-jsbn@~0.1.1: + version "0.1.2" + resolved "https://registry.npmmirror.com/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz#3a83a904e54353287874c564b7549386849a98c9" + integrity sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw== + dependencies: + jsbn "~0.1.0" + safer-buffer "^2.1.0" + +ee-first@1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" + integrity sha512-WMwm9LhRUo+WUaRN+vRuETqG89IgZphVSNkdFgeb6sS/E4OrDIN7t48CAewSHXc6C8lefD8KKfr5vY61brQlow== + +electron-to-chromium@^1.4.147: + version "1.4.156" + resolved "https://registry.npmmirror.com/electron-to-chromium/-/electron-to-chromium-1.4.156.tgz#fc398e1bfbe586135351ebfaf198473a82923af5" + integrity sha512-/Wj5NC7E0wHaMCdqxWz9B0lv7CcycDTiHyXCtbbu3pXM9TV2AOp8BtMqkVuqvJNdEvltBG6LxT2Q+BxY4LUCIA== + +emoji-regex@^7.0.1: + version "7.0.3" + resolved "https://registry.npmmirror.com/emoji-regex/-/emoji-regex-7.0.3.tgz#933a04052860c85e83c122479c4748a8e4c72156" + integrity sha512-CwBLREIQ7LvYFB0WyRvwhq5N5qPhc6PMjD6bYggFlI5YyDgl+0vxq5VHbMOFqLg7hfWzmu8T5Z1QofhmTIhItA== + +emoji-regex@^8.0.0: + version "8.0.0" + resolved "https://registry.npmmirror.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" + integrity sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A== + +emoji-regex@^9.2.2: + version "9.2.2" + resolved "https://registry.npmmirror.com/emoji-regex/-/emoji-regex-9.2.2.tgz#840c8803b0d8047f4ff0cf963176b32d4ef3ed72" + integrity sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg== + +emojis-list@^2.0.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/emojis-list/-/emojis-list-2.1.0.tgz#4daa4d9db00f9819880c79fa457ae5b09a1fd389" + integrity sha512-knHEZMgs8BB+MInokmNTg/OyPlAddghe1YBgNwJBc5zsJi/uyIcXoSDsL/W9ymOsBoBGdPIHXYJ9+qKFwRwDng== + +emojis-list@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/emojis-list/-/emojis-list-3.0.0.tgz#5570662046ad29e2e916e71aae260abdff4f6a78" + integrity sha512-/kyM18EfinwXZbno9FyUGeFh87KC8HRQBQGildHZbEuRyWFOmv1U10o9BBp8XVZDVNNuQKyIGIu5ZYAAXJ0V2Q== + +encodeurl@~1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/encodeurl/-/encodeurl-1.0.2.tgz#ad3ff4c86ec2d029322f5a02c3a9a606c95b3f59" + integrity sha512-TPJXq8JqFaVYm2CWmPvnP2Iyo4ZSM7/QKcSmuMLDObfpH5fi7RUGmd/rTDf+rut/saiDiQEeVTNgAmJEdAOx0w== + +end-of-stream@^1.1.0, end-of-stream@^1.4.1: + version "1.4.4" + resolved "https://registry.npmmirror.com/end-of-stream/-/end-of-stream-1.4.4.tgz#5ae64a5f45057baf3626ec14da0ca5e4b2431eb0" + integrity sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q== + dependencies: + once "^1.4.0" + +engine.io-client@~4.1.0: + version "4.1.4" + resolved "https://registry.npmmirror.com/engine.io-client/-/engine.io-client-4.1.4.tgz#0bda5ba4bd87bced2ad00b93c67e133d0fb981ba" + integrity sha512-843fqAdKeUMFqKi1sSjnR11tJ4wi8sIefu6+JC1OzkkJBmjtc/gM/rZ53tJfu5Iae/3gApm5veoS+v+gtT0+Fg== + dependencies: + base64-arraybuffer "0.1.4" + component-emitter "~1.3.0" + debug "~4.3.1" + engine.io-parser "~4.0.1" + has-cors "1.1.0" + parseqs "0.0.6" + parseuri "0.0.6" + ws "~7.4.2" + xmlhttprequest-ssl "~1.6.2" + yeast "0.1.2" + +engine.io-parser@~4.0.0, engine.io-parser@~4.0.1: + version "4.0.3" + resolved "https://registry.npmmirror.com/engine.io-parser/-/engine.io-parser-4.0.3.tgz#83d3a17acfd4226f19e721bb22a1ee8f7662d2f6" + integrity sha512-xEAAY0msNnESNPc00e19y5heTPX4y/TJ36gr8t1voOaNmTojP9b3oK3BbJLFufW2XFPQaaijpFewm2g2Um3uqA== + dependencies: + base64-arraybuffer "0.1.4" + +engine.io@~4.1.0: + version "4.1.2" + resolved "https://registry.npmmirror.com/engine.io/-/engine.io-4.1.2.tgz#f96ceb56d4b39cc7ca5bd29a20e9c99c1ad1a765" + integrity sha512-t5z6zjXuVLhXDMiFJPYsPOWEER8B0tIsD3ETgw19S1yg9zryvUfY3Vhtk3Gf4sihw/bQGIqQ//gjvVlu+Ca0bQ== + dependencies: + accepts "~1.3.4" + base64id "2.0.0" + cookie "~0.4.1" + cors "~2.8.5" + debug "~4.3.1" + engine.io-parser "~4.0.0" + ws "~7.4.2" + +enhanced-resolve@^5.8.3, enhanced-resolve@^5.9.3: + version "5.9.3" + resolved "https://registry.npmmirror.com/enhanced-resolve/-/enhanced-resolve-5.9.3.tgz#44a342c012cbc473254af5cc6ae20ebd0aae5d88" + integrity sha512-Bq9VSor+kjvW3f9/MiiR4eE3XYgOl7/rS8lnSxbRbF3kS0B2r+Y9w5krBWxZgDxASVZbdYrn5wT4j/Wb0J9qow== + dependencies: + graceful-fs "^4.2.4" + tapable "^2.2.0" + +enquirer@^2.3.5: + version "2.3.6" + resolved "https://registry.npmmirror.com/enquirer/-/enquirer-2.3.6.tgz#2a7fe5dd634a1e4125a975ec994ff5456dc3734d" + integrity sha512-yjNnPr315/FjS4zIsUxYguYUPP2e1NK4d7E7ZOLiyYCcbFBiTMyID+2wvm2w6+pZ/odMA7cRkjhsPbltwBOrLg== + dependencies: + ansi-colors "^4.1.1" + +entities@^2.0.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/entities/-/entities-2.2.0.tgz#098dc90ebb83d8dffa089d55256b351d34c4da55" + integrity sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A== + +entities@^4.2.0, entities@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/entities/-/entities-4.3.0.tgz#62915f08d67353bb4eb67e3d62641a4059aec656" + integrity sha512-/iP1rZrSEJ0DTlPiX+jbzlA3eVkY/e8L8SozroF395fIqE3TYF/Nz7YOMAawta+vLmyJ/hkGNNPcSbMADCCXbg== + +env-paths@^2.2.0: + version "2.2.1" + resolved "https://registry.npmmirror.com/env-paths/-/env-paths-2.2.1.tgz#420399d416ce1fbe9bc0a07c62fa68d67fd0f8f2" + integrity sha512-+h1lkLKhZMTYjog1VEpJNG7NZJWcuc2DDk/qsqSTRRCOXiLjeQ1d1/udrUGhqMxUgAlwKNZ0cf2uqan5GLuS2A== + +envinfo@^7.8.1: + version "7.8.1" + resolved "https://registry.npmmirror.com/envinfo/-/envinfo-7.8.1.tgz#06377e3e5f4d379fea7ac592d5ad8927e0c4d475" + integrity sha512-/o+BXHmB7ocbHEAs6F2EnG0ogybVVUdkRunTT2glZU9XAaGmhqskrvKwqXuDfNjEO0LZKWdejEEpnq8aM0tOaw== + +eol@^0.9.1: + version "0.9.1" + resolved "https://registry.npmmirror.com/eol/-/eol-0.9.1.tgz#f701912f504074be35c6117a5c4ade49cd547acd" + integrity sha512-Ds/TEoZjwggRoz/Q2O7SE3i4Jm66mqTDfmdHdq/7DKVk3bro9Q8h6WdXKdPqFLMoqxrDK5SVRzHVPOS6uuGtrg== + +error-ex@^1.3.1: + version "1.3.2" + resolved "https://registry.npmmirror.com/error-ex/-/error-ex-1.3.2.tgz#b4ac40648107fdcdcfae242f428bea8a14d4f1bf" + integrity sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g== + dependencies: + is-arrayish "^0.2.1" + +error-stack-parser@^2.0.6: + version "2.1.4" + resolved "https://registry.npmmirror.com/error-stack-parser/-/error-stack-parser-2.1.4.tgz#229cb01cdbfa84440bfa91876285b94680188286" + integrity sha512-Sk5V6wVazPhq5MhpO+AUxJn5x7XSXGl1R93Vn7i+zS15KDVxQijejNCrz8340/2bgLBjR9GtEG8ZVKONDjcqGQ== + dependencies: + stackframe "^1.3.4" + +es-abstract@^1.17.2, es-abstract@^1.19.0, es-abstract@^1.19.1, es-abstract@^1.19.2, es-abstract@^1.19.5, es-abstract@^1.20.1: + version "1.20.1" + resolved "https://registry.npmmirror.com/es-abstract/-/es-abstract-1.20.1.tgz#027292cd6ef44bd12b1913b828116f54787d1814" + integrity sha512-WEm2oBhfoI2sImeM4OF2zE2V3BYdSF+KnSi9Sidz51fQHd7+JuF8Xgcj9/0o+OWeIeIS/MiuNnlruQrJf16GQA== + dependencies: + call-bind "^1.0.2" + es-to-primitive "^1.2.1" + function-bind "^1.1.1" + function.prototype.name "^1.1.5" + get-intrinsic "^1.1.1" + get-symbol-description "^1.0.0" + has "^1.0.3" + has-property-descriptors "^1.0.0" + has-symbols "^1.0.3" + internal-slot "^1.0.3" + is-callable "^1.2.4" + is-negative-zero "^2.0.2" + is-regex "^1.1.4" + is-shared-array-buffer "^1.0.2" + is-string "^1.0.7" + is-weakref "^1.0.2" + object-inspect "^1.12.0" + object-keys "^1.1.1" + object.assign "^4.1.2" + regexp.prototype.flags "^1.4.3" + string.prototype.trimend "^1.0.5" + string.prototype.trimstart "^1.0.5" + unbox-primitive "^1.0.2" + +es-array-method-boxes-properly@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/es-array-method-boxes-properly/-/es-array-method-boxes-properly-1.0.0.tgz#873f3e84418de4ee19c5be752990b2e44718d09e" + integrity sha512-wd6JXUmyHmt8T5a2xreUwKcGPq6f1f+WwIJkijUqiGcJz1qqnZgP6XIK+QyIWU5lT7imeNxUll48bziG+TSYcA== + +es-module-lexer@^0.9.0: + version "0.9.3" + resolved "https://registry.npmmirror.com/es-module-lexer/-/es-module-lexer-0.9.3.tgz#6f13db00cc38417137daf74366f535c8eb438f19" + integrity sha512-1HQ2M2sPtxwnvOvT1ZClHyQDiggdNjURWpY2we6aMKCQiUVxTmVs2UYPLIrD84sS+kMdUwfBSylbJPwNnBrnHQ== + +es-shim-unscopables@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/es-shim-unscopables/-/es-shim-unscopables-1.0.0.tgz#702e632193201e3edf8713635d083d378e510241" + integrity sha512-Jm6GPcCdC30eMLbZ2x8z2WuRwAws3zTBBKuusffYVUrNj/GVSUAZ+xKMaUpfNDR5IbyNA5LJbaecoUVbmUcB1w== + dependencies: + has "^1.0.3" + +es-to-primitive@^1.2.1: + version "1.2.1" + resolved "https://registry.npmmirror.com/es-to-primitive/-/es-to-primitive-1.2.1.tgz#e55cd4c9cdc188bcefb03b366c736323fc5c898a" + integrity sha512-QCOllgZJtaUo9miYBcLChTUaHNjJF3PYs1VidD7AwiEj1kYxKeQTctLAezAOH5ZKRH0g2IgPn6KwB4IT8iRpvA== + dependencies: + is-callable "^1.1.4" + is-date-object "^1.0.1" + is-symbol "^1.0.2" + +es5-ext@^0.10.35, es5-ext@^0.10.46, es5-ext@^0.10.50, es5-ext@^0.10.53, es5-ext@~0.10.14, es5-ext@~0.10.2, es5-ext@~0.10.46: + version "0.10.61" + resolved "https://registry.npmmirror.com/es5-ext/-/es5-ext-0.10.61.tgz#311de37949ef86b6b0dcea894d1ffedb909d3269" + integrity sha512-yFhIqQAzu2Ca2I4SE2Au3rxVfmohU9Y7wqGR+s7+H7krk26NXhIRAZDgqd6xqjCEFUomDEA3/Bo/7fKmIkW1kA== + dependencies: + es6-iterator "^2.0.3" + es6-symbol "^3.1.3" + next-tick "^1.1.0" + +es6-iterator@2.0.3, es6-iterator@^2.0.3: + version "2.0.3" + resolved "https://registry.npmmirror.com/es6-iterator/-/es6-iterator-2.0.3.tgz#a7de889141a05a94b0854403b2d0a0fbfa98f3b7" + integrity sha512-zw4SRzoUkd+cl+ZoE15A9o1oQd920Bb0iOJMQkQhl3jNc03YqVjAhG7scf9C5KWRU/R13Orf588uCC6525o02g== + dependencies: + d "1" + es5-ext "^0.10.35" + es6-symbol "^3.1.1" + +es6-promise@^4.1.1: + version "4.2.8" + resolved "https://registry.npmmirror.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.npmmirror.com/es6-symbol/-/es6-symbol-3.1.3.tgz#bad5d3c1bcdac28269f4cb331e431c78ac705d18" + integrity sha512-NJ6Yn3FuDinBaBRWl/q5X/s4koRHBrgKAu+yGI6JCBeiu3qrcbJhwT2GeR/EXVfylRk8dpQVJoLEFhK+Mu31NA== + dependencies: + d "^1.0.1" + ext "^1.1.2" + +es6-weak-map@^2.0.3: + version "2.0.3" + resolved "https://registry.npmmirror.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" + +escalade@^3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/escalade/-/escalade-3.1.1.tgz#d8cfdc7000965c5a0174b4a82eaa5c0552742e40" + integrity sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw== + +escape-goat@^2.0.0: + version "2.1.1" + resolved "https://registry.npmmirror.com/escape-goat/-/escape-goat-2.1.1.tgz#1b2dc77003676c457ec760b2dc68edb648188675" + integrity sha512-8/uIhbG12Csjy2JEW7D9pHbreaVaS/OpN3ycnyvElTdwM5n6GY6W6e2IPemfvGZeUMqZ9A/3GqIZMgKnBhAw/Q== + +escape-html@~1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/escape-html/-/escape-html-1.0.3.tgz#0258eae4d3d0c0974de1c169188ef0051d1d1988" + integrity sha512-NiSupZ4OeuGwr68lGIeym/ksIZMJodUGOSCZ/FSnTxcrekbvqrgdUxlJOMpijaKZVjAJrWrGs/6Jy8OMuyj9ow== + +escape-string-regexp@^1.0.2, escape-string-regexp@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" + integrity sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg== + +escape-string-regexp@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz#a30304e99daa32e23b2fd20f51babd07cffca344" + integrity sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w== + +escape-string-regexp@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz#14ba83a5d373e3d311e5afca29cf5bfad965bf34" + integrity sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA== + +eslint-config-react-app@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/eslint-config-react-app/-/eslint-config-react-app-6.0.0.tgz#ccff9fc8e36b322902844cbd79197982be355a0e" + integrity sha512-bpoAAC+YRfzq0dsTk+6v9aHm/uqnDwayNAXleMypGl6CpxI9oXXscVHo4fk3eJPIn+rsbtNetB4r/ZIidFIE8A== + dependencies: + confusing-browser-globals "^1.0.10" + +eslint-import-resolver-node@^0.3.6: + version "0.3.6" + resolved "https://registry.npmmirror.com/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.6.tgz#4048b958395da89668252001dbd9eca6b83bacbd" + integrity sha512-0En0w03NRVMn9Uiyn8YRPDKvWjxCWkslUEhGNTdGx15RvPJYQ+lbOlqrlNI2vEAs4pDYK4f/HN2TbDmk5TP0iw== + dependencies: + debug "^3.2.7" + resolve "^1.20.0" + +eslint-module-utils@^2.7.3: + version "2.7.3" + resolved "https://registry.npmmirror.com/eslint-module-utils/-/eslint-module-utils-2.7.3.tgz#ad7e3a10552fdd0642e1e55292781bd6e34876ee" + integrity sha512-088JEC7O3lDZM9xGe0RerkOMd0EjFl+Yvd1jPWIkMT5u3H9+HC34mWWPnqPrN13gieT9pBOO+Qt07Nb/6TresQ== + dependencies: + debug "^3.2.7" + find-up "^2.1.0" + +eslint-plugin-flowtype@^5.10.0: + version "5.10.0" + resolved "https://registry.npmmirror.com/eslint-plugin-flowtype/-/eslint-plugin-flowtype-5.10.0.tgz#7764cc63940f215bf3f0bd2d9a1293b2b9b2b4bb" + integrity sha512-vcz32f+7TP+kvTUyMXZmCnNujBQZDNmcqPImw8b9PZ+16w1Qdm6ryRuYZYVaG9xRqqmAPr2Cs9FAX5gN+x/bjw== + dependencies: + lodash "^4.17.15" + string-natural-compare "^3.0.1" + +eslint-plugin-graphql@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/eslint-plugin-graphql/-/eslint-plugin-graphql-4.0.0.tgz#d238ff2baee4d632cfcbe787a7a70a1f50428358" + integrity sha512-d5tQm24YkVvCEk29ZR5ScsgXqAGCjKlMS8lx3mS7FS/EKsWbkvXQImpvic03EpMIvNTBW5e+2xnHzXB/VHNZJw== + dependencies: + "@babel/runtime" "^7.10.0" + graphql-config "^3.0.2" + lodash.flatten "^4.4.0" + lodash.without "^4.4.0" + +eslint-plugin-import@^2.26.0: + version "2.26.0" + resolved "https://registry.npmmirror.com/eslint-plugin-import/-/eslint-plugin-import-2.26.0.tgz#f812dc47be4f2b72b478a021605a59fc6fe8b88b" + integrity sha512-hYfi3FXaM8WPLf4S1cikh/r4IxnO6zrhZbEGz2b660EJRbuxgpDS5gkCuYgGWg2xxh2rBuIr4Pvhve/7c31koA== + dependencies: + array-includes "^3.1.4" + array.prototype.flat "^1.2.5" + debug "^2.6.9" + doctrine "^2.1.0" + eslint-import-resolver-node "^0.3.6" + eslint-module-utils "^2.7.3" + has "^1.0.3" + is-core-module "^2.8.1" + is-glob "^4.0.3" + minimatch "^3.1.2" + object.values "^1.1.5" + resolve "^1.22.0" + tsconfig-paths "^3.14.1" + +eslint-plugin-jsx-a11y@^6.5.1: + version "6.5.1" + resolved "https://registry.npmmirror.com/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.5.1.tgz#cdbf2df901040ca140b6ec14715c988889c2a6d8" + integrity sha512-sVCFKX9fllURnXT2JwLN5Qgo24Ug5NF6dxhkmxsMEUZhXRcGg+X3e1JbJ84YePQKBl5E0ZjAH5Q4rkdcGY99+g== + dependencies: + "@babel/runtime" "^7.16.3" + aria-query "^4.2.2" + array-includes "^3.1.4" + ast-types-flow "^0.0.7" + axe-core "^4.3.5" + axobject-query "^2.2.0" + damerau-levenshtein "^1.0.7" + emoji-regex "^9.2.2" + has "^1.0.3" + jsx-ast-utils "^3.2.1" + language-tags "^1.0.5" + minimatch "^3.0.4" + +eslint-plugin-react-hooks@^4.5.0: + version "4.6.0" + resolved "https://registry.npmmirror.com/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.6.0.tgz#4c3e697ad95b77e93f8646aaa1630c1ba607edd3" + integrity sha512-oFc7Itz9Qxh2x4gNHStv3BqJq54ExXmfC+a1NjAta66IAN87Wu0R/QArgIS9qKzX3dXKPI9H5crl9QchNMY9+g== + +eslint-plugin-react@^7.29.4: + version "7.30.0" + resolved "https://registry.npmmirror.com/eslint-plugin-react/-/eslint-plugin-react-7.30.0.tgz#8e7b1b2934b8426ac067a0febade1b13bd7064e3" + integrity sha512-RgwH7hjW48BleKsYyHK5vUAvxtE9SMPDKmcPRQgtRCYaZA0XQPt5FSkrU3nhz5ifzMZcA8opwmRJ2cmOO8tr5A== + dependencies: + array-includes "^3.1.5" + array.prototype.flatmap "^1.3.0" + doctrine "^2.1.0" + estraverse "^5.3.0" + jsx-ast-utils "^2.4.1 || ^3.0.0" + minimatch "^3.1.2" + object.entries "^1.1.5" + object.fromentries "^2.0.5" + object.hasown "^1.1.1" + object.values "^1.1.5" + prop-types "^15.8.1" + resolve "^2.0.0-next.3" + semver "^6.3.0" + string.prototype.matchall "^4.0.7" + +eslint-scope@5.1.1, eslint-scope@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/eslint-scope/-/eslint-scope-5.1.1.tgz#e786e59a66cb92b3f6c1fb0d508aab174848f48c" + integrity sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw== + dependencies: + esrecurse "^4.3.0" + estraverse "^4.1.1" + +eslint-utils@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/eslint-utils/-/eslint-utils-2.1.0.tgz#d2de5e03424e707dc10c74068ddedae708741b27" + integrity sha512-w94dQYoauyvlDc43XnGB8lU3Zt713vNChgt4EWwhXAP2XkBvndfxF0AgIqKOOasjPIPzj9JqgwkwbCYD0/V3Zg== + dependencies: + eslint-visitor-keys "^1.1.0" + +eslint-utils@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/eslint-utils/-/eslint-utils-3.0.0.tgz#8aebaface7345bb33559db0a1f13a1d2d48c3672" + integrity sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA== + dependencies: + eslint-visitor-keys "^2.0.0" + +eslint-visitor-keys@^1.1.0, eslint-visitor-keys@^1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/eslint-visitor-keys/-/eslint-visitor-keys-1.3.0.tgz#30ebd1ef7c2fdff01c3a4f151044af25fab0523e" + integrity sha512-6J72N8UNa462wa/KFODt/PJ3IU60SDpC3QXC1Hjc1BXXpfL2C9R5+AU7jhe0F6GREqVMh4Juu+NY7xn+6dipUQ== + +eslint-visitor-keys@^2.0.0, eslint-visitor-keys@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/eslint-visitor-keys/-/eslint-visitor-keys-2.1.0.tgz#f65328259305927392c938ed44eb0a5c9b2bd303" + integrity sha512-0rSmRBzXgDzIsD6mGdJgevzgezI534Cer5L/vyMX0kHzT/jiB43jRhd9YUlMGYLQy2zprNmoT8qasCGtY+QaKw== + +eslint-webpack-plugin@^2.6.0: + version "2.6.0" + resolved "https://registry.npmmirror.com/eslint-webpack-plugin/-/eslint-webpack-plugin-2.6.0.tgz#3bd4ada4e539cb1f6687d2f619073dbb509361cd" + integrity sha512-V+LPY/T3kur5QO3u+1s34VDTcRxjXWPUGM4hlmTb5DwVD0OQz631yGTxJZf4SpAqAjdbBVe978S8BJeHpAdOhQ== + dependencies: + "@types/eslint" "^7.28.2" + arrify "^2.0.1" + jest-worker "^27.3.1" + micromatch "^4.0.4" + normalize-path "^3.0.0" + schema-utils "^3.1.1" + +eslint@^7.32.0: + version "7.32.0" + resolved "https://registry.npmmirror.com/eslint/-/eslint-7.32.0.tgz#c6d328a14be3fb08c8d1d21e12c02fdb7a2a812d" + integrity sha512-VHZ8gX+EDfz+97jGcgyGCyRia/dPOd6Xh9yPv8Bl1+SoaIwD+a/vlrOmGRUyOYu7MwUhc7CxqeaDZU13S4+EpA== + dependencies: + "@babel/code-frame" "7.12.11" + "@eslint/eslintrc" "^0.4.3" + "@humanwhocodes/config-array" "^0.5.0" + ajv "^6.10.0" + chalk "^4.0.0" + cross-spawn "^7.0.2" + debug "^4.0.1" + doctrine "^3.0.0" + enquirer "^2.3.5" + escape-string-regexp "^4.0.0" + eslint-scope "^5.1.1" + eslint-utils "^2.1.0" + eslint-visitor-keys "^2.0.0" + espree "^7.3.1" + esquery "^1.4.0" + esutils "^2.0.2" + fast-deep-equal "^3.1.3" + file-entry-cache "^6.0.1" + functional-red-black-tree "^1.0.1" + glob-parent "^5.1.2" + globals "^13.6.0" + ignore "^4.0.6" + import-fresh "^3.0.0" + imurmurhash "^0.1.4" + is-glob "^4.0.0" + js-yaml "^3.13.1" + json-stable-stringify-without-jsonify "^1.0.1" + levn "^0.4.1" + lodash.merge "^4.6.2" + minimatch "^3.0.4" + natural-compare "^1.4.0" + optionator "^0.9.1" + progress "^2.0.0" + regexpp "^3.1.0" + semver "^7.2.1" + strip-ansi "^6.0.0" + strip-json-comments "^3.1.0" + table "^6.0.9" + text-table "^0.2.0" + v8-compile-cache "^2.0.3" + +espree@^7.3.0, espree@^7.3.1: + version "7.3.1" + resolved "https://registry.npmmirror.com/espree/-/espree-7.3.1.tgz#f2df330b752c6f55019f8bd89b7660039c1bbbb6" + integrity sha512-v3JCNCE64umkFpmkFGqzVKsOT0tN1Zr+ueqLZfpV1Ob8e+CEgPWa+OxCoGH3tnhimMKIaBm4m/vaRpJ/krRz2g== + dependencies: + acorn "^7.4.0" + acorn-jsx "^5.3.1" + eslint-visitor-keys "^1.3.0" + +esprima@^4.0.0: + version "4.0.1" + resolved "https://registry.npmmirror.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71" + integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A== + +esquery@^1.4.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/esquery/-/esquery-1.4.0.tgz#2148ffc38b82e8c7057dfed48425b3e61f0f24a5" + integrity sha512-cCDispWt5vHHtwMY2YrAQ4ibFkAL8RbH5YGBnZBc90MolvvfkkQcJro/aZiAQUlQ3qgrYS6D6v8Gc5G5CQsc9w== + dependencies: + estraverse "^5.1.0" + +esrecurse@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/esrecurse/-/esrecurse-4.3.0.tgz#7ad7964d679abb28bee72cec63758b1c5d2c9921" + integrity sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag== + dependencies: + estraverse "^5.2.0" + +estraverse@^4.1.1: + version "4.3.0" + resolved "https://registry.npmmirror.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d" + integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw== + +estraverse@^5.1.0, estraverse@^5.2.0, estraverse@^5.3.0: + version "5.3.0" + resolved "https://registry.npmmirror.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" + integrity sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA== + +esutils@^2.0.2: + version "2.0.3" + resolved "https://registry.npmmirror.com/esutils/-/esutils-2.0.3.tgz#74d2eb4de0b8da1293711910d50775b9b710ef64" + integrity sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g== + +etag@~1.8.1: + version "1.8.1" + resolved "https://registry.npmmirror.com/etag/-/etag-1.8.1.tgz#41ae2eeb65efa62268aebfea83ac7d79299b0887" + integrity sha512-aIL5Fx7mawVa300al2BnEE4iNvo1qETxLrPI/o05L7z6go7fCw1J6EQmbK4FmJ2AS7kgVF/KEZWufBfdClMcPg== + +event-emitter@^0.3.5: + version "0.3.5" + resolved "https://registry.npmmirror.com/event-emitter/-/event-emitter-0.3.5.tgz#df8c69eef1647923c7157b9ce83840610b02cc39" + integrity sha512-D9rRn9y7kLPnJ+hMq7S/nhvoKwwvVJahBi2BPmx3bvbsEdK3W9ii8cBSGjP+72/LnM4n6fo3+dkCX5FeTQruXA== + dependencies: + d "1" + es5-ext "~0.10.14" + +event-source-polyfill@1.0.25: + version "1.0.25" + resolved "https://registry.npmmirror.com/event-source-polyfill/-/event-source-polyfill-1.0.25.tgz#d8bb7f99cb6f8119c2baf086d9f6ee0514b6d9c8" + integrity sha512-hQxu6sN1Eq4JjoI7ITdQeGGUN193A2ra83qC0Ltm9I2UJVAten3OFVN6k5RX4YWeCS0BoC8xg/5czOCIHVosQg== + +event-target-shim@^5.0.0: + version "5.0.1" + resolved "https://registry.npmmirror.com/event-target-shim/-/event-target-shim-5.0.1.tgz#5d4d3ebdf9583d63a5333ce2deb7480ab2b05789" + integrity sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ== + +eventemitter3@^3.1.0: + version "3.1.2" + resolved "https://registry.npmmirror.com/eventemitter3/-/eventemitter3-3.1.2.tgz#2d3d48f9c346698fce83a85d7d664e98535df6e7" + integrity sha512-tvtQIeLVHjDkJYnzf2dgVMxfuSGJeM/7UCG17TT4EumTfNtF+0nebF/4zWOIkCreAbtNqhGEboB6BWrwqNaw4Q== + +eventemitter3@^4.0.0: + version "4.0.7" + resolved "https://registry.npmmirror.com/eventemitter3/-/eventemitter3-4.0.7.tgz#2de9b68f6528d5644ef5c59526a1b4a07306169f" + integrity sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw== + +events@^3.2.0: + version "3.3.0" + resolved "https://registry.npmmirror.com/events/-/events-3.3.0.tgz#31a95ad0a924e2d2c419a813aeb2c4e878ea7400" + integrity sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q== + +execa@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/execa/-/execa-1.0.0.tgz#c6236a5bb4df6d6f15e88e7f017798216749ddd8" + integrity sha512-adbxcyWV46qiHyvSp50TKt05tB4tK3HcmF7/nxfAdhnox83seTDbwnaqKO4sXRy7roHAIFqJP/Rw/AuEbX61LA== + dependencies: + cross-spawn "^6.0.0" + get-stream "^4.0.0" + is-stream "^1.1.0" + npm-run-path "^2.0.0" + p-finally "^1.0.0" + signal-exit "^3.0.0" + strip-eof "^1.0.0" + +execa@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/execa/-/execa-5.1.1.tgz#f80ad9cbf4298f7bd1d4c9555c21e93741c411dd" + integrity sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg== + dependencies: + cross-spawn "^7.0.3" + get-stream "^6.0.0" + human-signals "^2.1.0" + is-stream "^2.0.0" + merge-stream "^2.0.0" + npm-run-path "^4.0.1" + onetime "^5.1.2" + signal-exit "^3.0.3" + strip-final-newline "^2.0.0" + +exif-parser@^0.1.12: + version "0.1.12" + resolved "https://registry.npmmirror.com/exif-parser/-/exif-parser-0.1.12.tgz#58a9d2d72c02c1f6f02a0ef4a9166272b7760922" + integrity sha512-c2bQfLNbMzLPmzQuOr8fy0csy84WmwnER81W88DzTp9CYNPJ6yzOj2EZAh9pywYpqHnshVLHQJ8WzldAyfY+Iw== + +expand-template@^2.0.3: + version "2.0.3" + resolved "https://registry.npmmirror.com/expand-template/-/expand-template-2.0.3.tgz#6e14b3fcee0f3a6340ecb57d2e8918692052a47c" + integrity sha512-XYfuKMvj4O35f/pOXLObndIRvyQ+/+6AhODh+OKWj9S9498pHHn/IMszH+gt0fBCRWMNfk1ZSp5x3AifmnI2vg== + +express-graphql@^0.12.0: + version "0.12.0" + resolved "https://registry.npmmirror.com/express-graphql/-/express-graphql-0.12.0.tgz#58deabc309909ca2c9fe2f83f5fbe94429aa23df" + integrity sha512-DwYaJQy0amdy3pgNtiTDuGGM2BLdj+YO2SgbKoLliCfuHv3VVTt7vNG/ZqK2hRYjtYHE2t2KB705EU94mE64zg== + dependencies: + accepts "^1.3.7" + content-type "^1.0.4" + http-errors "1.8.0" + raw-body "^2.4.1" + +express-http-proxy@^1.6.3: + version "1.6.3" + resolved "https://registry.npmmirror.com/express-http-proxy/-/express-http-proxy-1.6.3.tgz#f3ef139ffd49a7962e7af0462bbcca557c913175" + integrity sha512-/l77JHcOUrDUX8V67E287VEUQT0lbm71gdGVoodnlWBziarYKgMcpqT7xvh/HM8Jv52phw8Bd8tY+a7QjOr7Yg== + dependencies: + debug "^3.0.1" + es6-promise "^4.1.1" + raw-body "^2.3.0" + +express@^4.17.1: + version "4.18.1" + resolved "https://registry.npmmirror.com/express/-/express-4.18.1.tgz#7797de8b9c72c857b9cd0e14a5eea80666267caf" + integrity sha512-zZBcOX9TfehHQhtupq57OF8lFZ3UZi08Y97dwFCkD8p9d/d2Y3M+ykKcwaMDEL+4qyUolgBDX6AblpR3fL212Q== + dependencies: + accepts "~1.3.8" + array-flatten "1.1.1" + body-parser "1.20.0" + content-disposition "0.5.4" + content-type "~1.0.4" + cookie "0.5.0" + cookie-signature "1.0.6" + debug "2.6.9" + depd "2.0.0" + encodeurl "~1.0.2" + escape-html "~1.0.3" + etag "~1.8.1" + finalhandler "1.2.0" + fresh "0.5.2" + http-errors "2.0.0" + merge-descriptors "1.0.1" + methods "~1.1.2" + on-finished "2.4.1" + parseurl "~1.3.3" + path-to-regexp "0.1.7" + proxy-addr "~2.0.7" + qs "6.10.3" + range-parser "~1.2.1" + safe-buffer "5.2.1" + send "0.18.0" + serve-static "1.15.0" + setprototypeof "1.2.0" + statuses "2.0.1" + type-is "~1.6.18" + utils-merge "1.0.1" + vary "~1.1.2" + +ext@^1.1.2: + version "1.6.0" + resolved "https://registry.npmmirror.com/ext/-/ext-1.6.0.tgz#3871d50641e874cc172e2b53f919842d19db4c52" + integrity sha512-sdBImtzkq2HpkdRLtlLWDa6w4DX22ijZLKx8BMPUuKe1c5lbN6xwQDQCxSfxBQnHZ13ls/FH0MQZx/q/gr6FQg== + dependencies: + type "^2.5.0" + +extend-shallow@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/extend-shallow/-/extend-shallow-2.0.1.tgz#51af7d614ad9a9f610ea1bafbb989d6b1c56890f" + integrity sha512-zCnTtlxNoAiDc3gqY2aYAWFx7XWWiasuF2K8Me5WbN8otHKTUKBwjPtNpRs/rbUZm7KxWAaNj7P1a/p52GbVug== + dependencies: + is-extendable "^0.1.0" + +extend@^3.0.0, extend@^3.0.2, extend@~3.0.2: + version "3.0.2" + resolved "https://registry.npmmirror.com/extend/-/extend-3.0.2.tgz#f8b1136b4071fbd8eb140aff858b1019ec2915fa" + integrity sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g== + +external-editor@^3.0.3: + version "3.1.0" + resolved "https://registry.npmmirror.com/external-editor/-/external-editor-3.1.0.tgz#cb03f740befae03ea4d283caed2741a83f335495" + integrity sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew== + dependencies: + chardet "^0.7.0" + iconv-lite "^0.4.24" + tmp "^0.0.33" + +extract-files@9.0.0: + version "9.0.0" + resolved "https://registry.npmmirror.com/extract-files/-/extract-files-9.0.0.tgz#8a7744f2437f81f5ed3250ed9f1550de902fe54a" + integrity sha512-CvdFfHkC95B4bBBk36hcEmvdR2awOdhhVUYH6S/zrVj3477zven/fJMYg7121h4T1xHZC+tetUpubpAhxwI7hQ== + +extsprintf@1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/extsprintf/-/extsprintf-1.3.0.tgz#96918440e3041a7a414f8c52e3c574eb3c3e1e05" + integrity sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g== + +extsprintf@^1.2.0: + version "1.4.1" + resolved "https://registry.npmmirror.com/extsprintf/-/extsprintf-1.4.1.tgz#8d172c064867f235c0c84a596806d279bf4bcc07" + integrity sha512-Wrk35e8ydCKDj/ArClo1VrPVmN8zph5V4AtHwIuHhvMXsKf73UT3BOD+azBIW+3wOJ4FhEH7zyaJCFvChjYvMA== + +fast-deep-equal@^3.1.1, fast-deep-equal@^3.1.3: + version "3.1.3" + resolved "https://registry.npmmirror.com/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz#3a7d56b559d6cbc3eb512325244e619a65c6c525" + integrity sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q== + +fast-glob@^3.1.1, fast-glob@^3.2.11, fast-glob@^3.2.9: + version "3.2.11" + resolved "https://registry.npmmirror.com/fast-glob/-/fast-glob-3.2.11.tgz#a1172ad95ceb8a16e20caa5c5e56480e5129c1d9" + integrity sha512-xrO3+1bxSo3ZVHAnqzyuewYT6aMFHRAd4Kcs92MAonjwQZLsK9d0SF1IyQ3k5PoirxTW0Oe/RqFgMQ6TcNE5Ew== + dependencies: + "@nodelib/fs.stat" "^2.0.2" + "@nodelib/fs.walk" "^1.2.3" + glob-parent "^5.1.2" + merge2 "^1.3.0" + micromatch "^4.0.4" + +fast-json-stable-stringify@^2.0.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz#874bf69c6f404c2b5d99c481341399fd55892633" + integrity sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw== + +fast-levenshtein@^2.0.6: + version "2.0.6" + resolved "https://registry.npmmirror.com/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz#3d8a5c66883a16a30ca8643e851f19baa7797917" + integrity sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw== + +fastest-levenshtein@^1.0.12: + version "1.0.12" + resolved "https://registry.npmmirror.com/fastest-levenshtein/-/fastest-levenshtein-1.0.12.tgz#9990f7d3a88cc5a9ffd1f1745745251700d497e2" + integrity sha512-On2N+BpYJ15xIC974QNVuYGMOlEVt4s0EOI3wwMqOmK1fdDY+FN/zltPV8vosq4ad4c/gJ1KHScUn/6AWIgiow== + +fastq@^1.13.0, fastq@^1.6.0: + version "1.13.0" + resolved "https://registry.npmmirror.com/fastq/-/fastq-1.13.0.tgz#616760f88a7526bdfc596b7cab8c18938c36b98c" + integrity sha512-YpkpUnK8od0o1hmeSc7UUs/eB/vIPWJYjKck2QKIzAf71Vm1AAQ3EbuZB3g2JIy+pg+ERD0vqI79KyZiB2e2Nw== + dependencies: + reusify "^1.0.4" + +fb-watchman@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/fb-watchman/-/fb-watchman-2.0.1.tgz#fc84fb39d2709cf3ff6d743706157bb5708a8a85" + integrity sha512-DkPJKQeY6kKwmuMretBhr7G6Vodr7bFwDYTXIkfG1gjvNpaxBTQV3PbXg6bR1c1UP4jPOX0jHUbbHANL9vRjVg== + dependencies: + bser "2.1.1" + +fbemitter@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/fbemitter/-/fbemitter-3.0.0.tgz#00b2a1af5411254aab416cd75f9e6289bee4bff3" + integrity sha512-KWKaceCwKQU0+HPoop6gn4eOHk50bBv/VxjJtGMfwmJt3D29JpN4H4eisCtIPA+a8GVBam+ldMMpMjJUvpDyHw== + dependencies: + fbjs "^3.0.0" + +fbjs-css-vars@^1.0.0: + version "1.0.2" + resolved "https://registry.npmmirror.com/fbjs-css-vars/-/fbjs-css-vars-1.0.2.tgz#216551136ae02fe255932c3ec8775f18e2c078b8" + integrity sha512-b2XGFAFdWZWg0phtAWLHCk836A1Xann+I+Dgd3Gk64MHKZO44FfoD1KxyvbSh0qZsIoXQGGlVztIY+oitJPpRQ== + +fbjs@^3.0.0, fbjs@^3.0.1: + version "3.0.4" + resolved "https://registry.npmmirror.com/fbjs/-/fbjs-3.0.4.tgz#e1871c6bd3083bac71ff2da868ad5067d37716c6" + integrity sha512-ucV0tDODnGV3JCnnkmoszb5lf4bNpzjv80K41wd4k798Etq+UYD0y0TIfalLjZoKgjive6/adkRnszwapiDgBQ== + dependencies: + cross-fetch "^3.1.5" + fbjs-css-vars "^1.0.0" + loose-envify "^1.0.0" + object-assign "^4.1.0" + promise "^7.1.1" + setimmediate "^1.0.5" + ua-parser-js "^0.7.30" + +fd@~0.0.2: + version "0.0.3" + resolved "https://registry.npmmirror.com/fd/-/fd-0.0.3.tgz#b3240de86dbf5a345baae7382a07d4713566ff0c" + integrity sha512-iAHrIslQb3U68OcMSP0kkNWabp7sSN6d2TBSb2JO3gcLJVDd4owr/hKM4SFJovFOUeeXeItjYgouEDTMWiVAnA== + +figures@^3.0.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/figures/-/figures-3.2.0.tgz#625c18bd293c604dc4a8ddb2febf0c88341746af" + integrity sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg== + dependencies: + escape-string-regexp "^1.0.5" + +file-entry-cache@^6.0.1: + version "6.0.1" + resolved "https://registry.npmmirror.com/file-entry-cache/-/file-entry-cache-6.0.1.tgz#211b2dd9659cb0394b073e7323ac3c933d522027" + integrity sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg== + dependencies: + flat-cache "^3.0.4" + +file-loader@^6.2.0: + version "6.2.0" + resolved "https://registry.npmmirror.com/file-loader/-/file-loader-6.2.0.tgz#baef7cf8e1840df325e4390b4484879480eebe4d" + integrity sha512-qo3glqyTa61Ytg4u73GultjHGjdRyig3tG6lPtyX/jOEJvHif9uB0/OCI2Kif6ctF3caQTW2G5gym21oAsI4pw== + dependencies: + loader-utils "^2.0.0" + schema-utils "^3.0.0" + +file-type@^16.5.3: + version "16.5.3" + resolved "https://registry.npmmirror.com/file-type/-/file-type-16.5.3.tgz#474b7e88c74724046abb505e9b8ed4db30c4fc06" + integrity sha512-uVsl7iFhHSOY4bEONLlTK47iAHtNsFHWP5YE4xJfZ4rnX7S1Q3wce09XgqSC7E/xh8Ncv/be1lNoyprlUH/x6A== + dependencies: + readable-web-to-node-stream "^3.0.0" + strtok3 "^6.2.4" + token-types "^4.1.1" + +file-type@^9.0.0: + version "9.0.0" + resolved "https://registry.npmmirror.com/file-type/-/file-type-9.0.0.tgz#a68d5ad07f486414dfb2c8866f73161946714a18" + integrity sha512-Qe/5NJrgIOlwijpq3B7BEpzPFcgzggOTagZmkXQY4LA6bsXKTUstK7Wp12lEJ/mLKTpvIZxmIuRcLYWT6ov9lw== + +filename-reserved-regex@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/filename-reserved-regex/-/filename-reserved-regex-2.0.0.tgz#abf73dfab735d045440abfea2d91f389ebbfa229" + integrity sha512-lc1bnsSr4L4Bdif8Xb/qrtokGbq5zlsms/CYH8PP+WtCkGNF65DPiQY8vG3SakEdRn8Dlnm+gW/qWKKjS5sZzQ== + +filenamify@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/filenamify/-/filenamify-4.3.0.tgz#62391cb58f02b09971c9d4f9d63b3cf9aba03106" + integrity sha512-hcFKyUG57yWGAzu1CMt/dPzYZuv+jAJUT85bL8mrXvNe6hWj6yEHEc4EdcgiA6Z3oi1/9wXJdZPXF2dZNgwgOg== + dependencies: + filename-reserved-regex "^2.0.0" + strip-outer "^1.0.1" + trim-repeated "^1.0.0" + +filesize@^8.0.6: + version "8.0.7" + resolved "https://registry.npmmirror.com/filesize/-/filesize-8.0.7.tgz#695e70d80f4e47012c132d57a059e80c6b580bd8" + integrity sha512-pjmC+bkIF8XI7fWaH8KxHcZL3DPybs1roSKP4rKDvy20tAWwIObE4+JIseG2byfGKhud5ZnM4YSGKBz7Sh0ndQ== + +fill-range@^7.0.1: + version "7.0.1" + resolved "https://registry.npmmirror.com/fill-range/-/fill-range-7.0.1.tgz#1919a6a7c75fe38b2c7c77e5198535da9acdda40" + integrity sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ== + dependencies: + to-regex-range "^5.0.1" + +filter-obj@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/filter-obj/-/filter-obj-1.1.0.tgz#9b311112bc6c6127a16e016c6c5d7f19e0805c5b" + integrity sha512-8rXg1ZnX7xzy2NGDVkBVaAy+lSlPNwad13BtgSlLuxfIslyt5Vg64U7tFcCt4WS1R0hvtnQybT/IyCkGZ3DpXQ== + +finalhandler@1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/finalhandler/-/finalhandler-1.2.0.tgz#7d23fe5731b207b4640e4fcd00aec1f9207a7b32" + integrity sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg== + dependencies: + debug "2.6.9" + encodeurl "~1.0.2" + escape-html "~1.0.3" + on-finished "2.4.1" + parseurl "~1.3.3" + statuses "2.0.1" + unpipe "~1.0.0" + +find-cache-dir@^3.3.1, find-cache-dir@^3.3.2: + version "3.3.2" + resolved "https://registry.npmmirror.com/find-cache-dir/-/find-cache-dir-3.3.2.tgz#b30c5b6eff0730731aea9bbd9dbecbd80256d64b" + integrity sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig== + dependencies: + commondir "^1.0.1" + make-dir "^3.0.2" + pkg-dir "^4.1.0" + +find-up@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/find-up/-/find-up-2.1.0.tgz#45d1b7e506c717ddd482775a2b77920a3c0c57a7" + integrity sha512-NWzkk0jSJtTt08+FBFMvXoeZnOJD+jTtsRmBYbAIzJdX6l7dLgR7CTubCM5/eDdPUBvLCeVasP1brfVR/9/EZQ== + dependencies: + locate-path "^2.0.0" + +find-up@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/find-up/-/find-up-3.0.0.tgz#49169f1d7993430646da61ecc5ae355c21c97b73" + integrity sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg== + dependencies: + locate-path "^3.0.0" + +find-up@^4.0.0, find-up@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" + integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== + dependencies: + locate-path "^5.0.0" + path-exists "^4.0.0" + +find-up@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/find-up/-/find-up-5.0.0.tgz#4c92819ecb7083561e4f4a240a86be5198f536fc" + integrity sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng== + dependencies: + locate-path "^6.0.0" + path-exists "^4.0.0" + +flat-cache@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/flat-cache/-/flat-cache-3.0.4.tgz#61b0338302b2fe9f957dcc32fc2a87f1c3048b11" + integrity sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg== + dependencies: + flatted "^3.1.0" + rimraf "^3.0.2" + +flatted@^3.1.0: + version "3.2.5" + resolved "https://registry.npmmirror.com/flatted/-/flatted-3.2.5.tgz#76c8584f4fc843db64702a6bd04ab7a8bd666da3" + integrity sha512-WIWGi2L3DyTUvUrwRKgGi9TwxQMUEqPOPQBVi71R96jZXJdFskXEmf54BoZaS1kknGODoIGASGEzBUYdyMCBJg== + +flux@^4.0.1: + version "4.0.3" + resolved "https://registry.npmmirror.com/flux/-/flux-4.0.3.tgz#573b504a24982c4768fdfb59d8d2ea5637d72ee7" + integrity sha512-yKAbrp7JhZhj6uiT1FTuVMlIAT1J4jqEyBpFApi1kxpGZCvacMVc/t1pMQyotqHhAgvoE3bNvAykhCo2CLjnYw== + dependencies: + fbemitter "^3.0.0" + fbjs "^3.0.1" + +follow-redirects@^1.0.0, follow-redirects@^1.14.0: + version "1.15.1" + resolved "https://registry.npmmirror.com/follow-redirects/-/follow-redirects-1.15.1.tgz#0ca6a452306c9b276e4d3127483e29575e207ad5" + integrity sha512-yLAMQs+k0b2m7cVxpS1VKJVvoz7SS9Td1zss3XRwXj+ZDH00RJgnuLx7E44wx02kQLrdM3aOOy+FpzS7+8OizA== + +forever-agent@~0.6.1: + version "0.6.1" + resolved "https://registry.npmmirror.com/forever-agent/-/forever-agent-0.6.1.tgz#fbc71f0c41adeb37f96c577ad1ed42d8fdacca91" + integrity sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw== + +fork-ts-checker-webpack-plugin@^6.5.0: + version "6.5.2" + resolved "https://registry.npmmirror.com/fork-ts-checker-webpack-plugin/-/fork-ts-checker-webpack-plugin-6.5.2.tgz#4f67183f2f9eb8ba7df7177ce3cf3e75cdafb340" + integrity sha512-m5cUmF30xkZ7h4tWUgTAcEaKmUW7tfyUyTqNNOz7OxWJ0v1VWKTcOvH8FWHUwSjlW/356Ijc9vi3XfcPstpQKA== + dependencies: + "@babel/code-frame" "^7.8.3" + "@types/json-schema" "^7.0.5" + chalk "^4.1.0" + chokidar "^3.4.2" + cosmiconfig "^6.0.0" + deepmerge "^4.2.2" + fs-extra "^9.0.0" + glob "^7.1.6" + memfs "^3.1.2" + minimatch "^3.0.4" + schema-utils "2.7.0" + semver "^7.3.2" + tapable "^1.0.0" + +form-data@4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/form-data/-/form-data-4.0.0.tgz#93919daeaf361ee529584b9b31664dc12c9fa452" + integrity sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww== + dependencies: + asynckit "^0.4.0" + combined-stream "^1.0.8" + mime-types "^2.1.12" + +form-data@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/form-data/-/form-data-3.0.1.tgz#ebd53791b78356a99af9a300d4282c4d5eb9755f" + integrity sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg== + dependencies: + asynckit "^0.4.0" + combined-stream "^1.0.8" + mime-types "^2.1.12" + +form-data@~2.3.2: + version "2.3.3" + resolved "https://registry.npmmirror.com/form-data/-/form-data-2.3.3.tgz#dcce52c05f644f298c6a7ab936bd724ceffbf3a6" + integrity sha512-1lLKB2Mu3aGP1Q/2eCOx0fNbRMe7XdwktwOruhfqqd0rIJWwN4Dh+E3hrPSlDCXnSR7UtZ1N38rVXm+6+MEhJQ== + dependencies: + asynckit "^0.4.0" + combined-stream "^1.0.6" + mime-types "^2.1.12" + +forwarded@0.2.0: + version "0.2.0" + resolved "https://registry.npmmirror.com/forwarded/-/forwarded-0.2.0.tgz#2269936428aad4c15c7ebe9779a84bf0b2a81811" + integrity sha512-buRG0fpBtRHSTCOASe6hD258tEubFoRLb4ZNA6NxMVHNw2gOcwHo9wyablzMzOA5z9xA9L1KNjk/Nt6MT9aYow== + +fraction.js@^4.2.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/fraction.js/-/fraction.js-4.2.0.tgz#448e5109a313a3527f5a3ab2119ec4cf0e0e2950" + integrity sha512-MhLuK+2gUcnZe8ZHlaaINnQLl0xRIGRfcGk2yl8xoQAfHrSsL3rYu6FCmBdkdbhc9EPlwyGHewaRsvwRMJtAlA== + +fresh@0.5.2: + version "0.5.2" + resolved "https://registry.npmmirror.com/fresh/-/fresh-0.5.2.tgz#3d8cadd90d976569fa835ab1f8e4b23a105605a7" + integrity sha512-zJ2mQYM18rEFOudeV4GShTGIQ7RbzA7ozbU9I/XBpm7kqgMywgmylMwXHxZJmkVoYkna9d2pVXVXPdYTP9ej8Q== + +fs-constants@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/fs-constants/-/fs-constants-1.0.0.tgz#6be0de9be998ce16af8afc24497b9ee9b7ccd9ad" + integrity sha512-y6OAwoSIf7FyjMIv94u+b5rdheZEjzR63GTyZJm5qh4Bi+2YgwLCcI/fPFZkL5PSixOt6ZNKm+w+Hfp/Bciwow== + +fs-exists-cached@1.0.0, fs-exists-cached@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/fs-exists-cached/-/fs-exists-cached-1.0.0.tgz#cf25554ca050dc49ae6656b41de42258989dcbce" + integrity sha512-kSxoARUDn4F2RPXX48UXnaFKwVU7Ivd/6qpzZL29MCDmr9sTvybv4gFCp+qaI4fM9m0z9fgz/yJvi56GAz+BZg== + +fs-extra@^10.1.0: + version "10.1.0" + resolved "https://registry.npmmirror.com/fs-extra/-/fs-extra-10.1.0.tgz#02873cfbc4084dde127eaa5f9905eef2325d1abf" + integrity sha512-oRXApq54ETRj4eMiFzGnHWGy+zo5raudjuxN0b8H7s/RU2oW0Wvsx9O0ACRN/kRq9E8Vu/ReskGB5o3ji+FzHQ== + dependencies: + graceful-fs "^4.2.0" + jsonfile "^6.0.1" + universalify "^2.0.0" + +fs-extra@^4.0.2: + version "4.0.3" + resolved "https://registry.npmmirror.com/fs-extra/-/fs-extra-4.0.3.tgz#0d852122e5bc5beb453fb028e9c0c9bf36340c94" + integrity sha512-q6rbdDd1o2mAnQreO7YADIxf/Whx4AHBiRf6d+/cVT8h44ss+lHgxf1FemcqDnQt9X3ct4McHr+JMGlYSsK7Cg== + dependencies: + graceful-fs "^4.1.2" + jsonfile "^4.0.0" + universalify "^0.1.0" + +fs-extra@^9.0.0: + version "9.1.0" + resolved "https://registry.npmmirror.com/fs-extra/-/fs-extra-9.1.0.tgz#5954460c764a8da2094ba3554bf839e6b9a7c86d" + integrity sha512-hcg3ZmepS30/7BSFqRvoo3DOMQu7IjqxO5nCDt+zM9XWjb33Wg7ziNT+Qvqbuc3+gWpzO02JubVyk2G4Zvo1OQ== + dependencies: + at-least-node "^1.0.0" + graceful-fs "^4.2.0" + jsonfile "^6.0.1" + universalify "^2.0.0" + +fs-minipass@^2.0.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/fs-minipass/-/fs-minipass-2.1.0.tgz#7f5036fdbf12c63c169190cbe4199c852271f9fb" + integrity sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg== + dependencies: + minipass "^3.0.0" + +fs-monkey@1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/fs-monkey/-/fs-monkey-1.0.3.tgz#ae3ac92d53bb328efe0e9a1d9541f6ad8d48e2d3" + integrity sha512-cybjIfiiE+pTWicSCLFHSrXZ6EilF30oh91FDP9S2B051prEa7QWfrVTQm10/dDpswBDXZugPa1Ogu8Yh+HV0Q== + +fs.realpath@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" + integrity sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw== + +fsevents@~2.3.2: + version "2.3.2" + resolved "https://registry.npmmirror.com/fsevents/-/fsevents-2.3.2.tgz#8a526f78b8fdf4623b709e0b975c52c24c02fd1a" + integrity sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA== + +function-bind@^1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d" + integrity sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A== + +function.prototype.name@^1.1.5: + version "1.1.5" + resolved "https://registry.npmmirror.com/function.prototype.name/-/function.prototype.name-1.1.5.tgz#cce0505fe1ffb80503e6f9e46cc64e46a12a9621" + integrity sha512-uN7m/BzVKQnCUF/iW8jYea67v++2u7m5UgENbHRtdDVclOUP+FMPlCNdmk0h/ysGyo2tavMJEDqJAkJdRa1vMA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.0" + functions-have-names "^1.2.2" + +functional-red-black-tree@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz#1b0ab3bd553b2a0d6399d29c0e3ea0b252078327" + integrity sha512-dsKNQNdj6xA3T+QlADDA7mOSlX0qiMINjn0cgr+eGHGsbSHzTabcIogz2+p/iqP1Xs6EP/sS2SbqH+brGTbq0g== + +functions-have-names@^1.2.2: + version "1.2.3" + resolved "https://registry.npmmirror.com/functions-have-names/-/functions-have-names-1.2.3.tgz#0404fe4ee2ba2f607f0e0ec3c80bae994133b834" + integrity sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ== + +gatsby-cli@^4.17.1: + version "4.17.1" + resolved "https://registry.npmmirror.com/gatsby-cli/-/gatsby-cli-4.17.1.tgz#e7fc847b6cad9580012aaac9dc79e5a1b6275630" + integrity sha512-yLNRGKo8vyeiqKNTH5MkE0Q3AF6Un1v3MVlXzvoyRNs1DIHwkkZ22jGOB64/KIIR/o2cjRsrKJNCVjGV7qNnzw== + dependencies: + "@babel/code-frame" "^7.14.0" + "@babel/core" "^7.15.5" + "@babel/generator" "^7.16.8" + "@babel/helper-plugin-utils" "^7.16.7" + "@babel/preset-typescript" "^7.16.7" + "@babel/runtime" "^7.15.4" + "@babel/template" "^7.16.7" + "@babel/types" "^7.16.8" + "@jridgewell/trace-mapping" "^0.3.13" + "@types/common-tags" "^1.8.1" + better-opn "^2.1.1" + boxen "^5.1.2" + chalk "^4.1.2" + clipboardy "^2.3.0" + common-tags "^1.8.2" + configstore "^5.0.1" + convert-hrtime "^3.0.0" + create-gatsby "^2.17.0" + envinfo "^7.8.1" + execa "^5.1.1" + fs-exists-cached "^1.0.0" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + gatsby-telemetry "^3.17.0" + hosted-git-info "^3.0.8" + is-valid-path "^0.1.1" + joi "^17.4.2" + lodash "^4.17.21" + meant "^1.0.3" + node-fetch "^2.6.6" + opentracing "^0.14.5" + pretty-error "^2.1.2" + progress "^2.0.3" + prompts "^2.4.2" + redux "4.1.2" + resolve-cwd "^3.0.0" + semver "^7.3.7" + signal-exit "^3.0.6" + stack-trace "^0.0.10" + strip-ansi "^6.0.1" + update-notifier "^5.1.0" + yargs "^15.4.1" + yoga-layout-prebuilt "^1.10.0" + yurnalist "^2.1.0" + +gatsby-core-utils@^3.16.0: + version "3.16.0" + resolved "https://registry.npmmirror.com/gatsby-core-utils/-/gatsby-core-utils-3.16.0.tgz#055e250614221b77168d0ca507a56a87d2cdbf35" + integrity sha512-cr3aIlzfzQkXSKng0dfAkg+v+Q0WOFJ1CCpM6HFvMykYtw5vSdaLxs6gwNmtPowG1wmCHkQ+pcyqppP+IdmVsg== + dependencies: + "@babel/runtime" "^7.15.4" + ci-info "2.0.0" + configstore "^5.0.1" + fastq "^1.13.0" + file-type "^16.5.3" + fs-extra "^10.1.0" + got "^11.8.3" + import-from "^4.0.0" + lmdb "2.3.10" + lock "^1.1.0" + node-object-hash "^2.3.10" + proper-lockfile "^4.1.2" + resolve-from "^5.0.0" + tmp "^0.2.1" + xdg-basedir "^4.0.0" + +gatsby-core-utils@^3.17.0: + version "3.17.0" + resolved "https://registry.npmmirror.com/gatsby-core-utils/-/gatsby-core-utils-3.17.0.tgz#5fe33c4267a3d0a7439b8065ee9c16f5cc2aead1" + integrity sha512-1e0YaqTAEpSSBkpWkY703lu+Bl76ASXUvUcpnNO3CavCYZsRQxAXtMXIKIEvhm1z6zWJmY9HILo6/DjP+PHeyw== + dependencies: + "@babel/runtime" "^7.15.4" + ci-info "2.0.0" + configstore "^5.0.1" + fastq "^1.13.0" + file-type "^16.5.3" + fs-extra "^10.1.0" + got "^11.8.3" + import-from "^4.0.0" + lmdb "2.5.2" + lock "^1.1.0" + node-object-hash "^2.3.10" + proper-lockfile "^4.1.2" + resolve-from "^5.0.0" + tmp "^0.2.1" + xdg-basedir "^4.0.0" + +gatsby-graphiql-explorer@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/gatsby-graphiql-explorer/-/gatsby-graphiql-explorer-2.17.0.tgz#128a4dc46ae48c31d858b0c9524344ec6904cb43" + integrity sha512-IWAcxMlOofpg8wRzLGjuey3HunWu3bLilKurX8csD0d+YsuiB4aqZc7dWi2Zc15ZHZ5Hh8zMpOv0MVW9qhjnCg== + dependencies: + "@babel/runtime" "^7.15.4" + +gatsby-legacy-polyfills@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/gatsby-legacy-polyfills/-/gatsby-legacy-polyfills-2.17.0.tgz#f8ff869e0aa95503b58a397c8ca538cc67487031" + integrity sha512-tpONSDutxEZY3vOdUaZ1yE9d0QWZ5/p47juvnhTv0qjGkBwow3gfCpgJwDzWyzKl0w0geesZwlLnaCA3WSEv7Q== + dependencies: + "@babel/runtime" "^7.15.4" + core-js-compat "3.9.0" + +gatsby-link@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-link/-/gatsby-link-4.17.0.tgz#cfccc7c8d702da9592a4c6172de03857af6e39fb" + integrity sha512-hDh+9OFMHNQ/5YdDdTcpSMB8xAYSP4OMkRuwmgPKH1jtOuov3pzwFAo19RR34CsOqzKd3LSQbCjIovOksjlAeA== + dependencies: + "@babel/runtime" "^7.15.4" + "@types/reach__router" "^1.3.10" + gatsby-page-utils "^2.17.0" + prop-types "^15.8.1" + +gatsby-page-utils@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/gatsby-page-utils/-/gatsby-page-utils-2.17.0.tgz#8f8cdb48493b9667b927482804b75be8405baea7" + integrity sha512-aeDrvYm6uTXayr/Yw0dNQUy+P9a5TshuG9UnRddCB7jQAR1aufjkNu8WH93l151p4jqUoYGemxgZIhrNv1+i/g== + dependencies: + "@babel/runtime" "^7.15.4" + bluebird "^3.7.2" + chokidar "^3.5.2" + fs-exists-cached "^1.0.0" + gatsby-core-utils "^3.17.0" + glob "^7.2.3" + lodash "^4.17.21" + micromatch "^4.0.5" + +gatsby-parcel-config@0.8.0: + version "0.8.0" + resolved "https://registry.npmmirror.com/gatsby-parcel-config/-/gatsby-parcel-config-0.8.0.tgz#4020ac11abe89d0b1bf19db0b78b7e0718170100" + integrity sha512-HzLU8uoJLuakH08T27K8GKx7rcLEVkKVClffAuVKrlcVYhNH+x1LvIwe+uMTIIdfu+YtUpUP1PpTdua6YfrVTQ== + dependencies: + "@gatsbyjs/parcel-namer-relative-to-cwd" "^1.2.0" + "@parcel/bundler-default" "2.6.0" + "@parcel/compressor-raw" "2.6.0" + "@parcel/namer-default" "2.6.0" + "@parcel/optimizer-terser" "2.6.0" + "@parcel/packager-js" "2.6.0" + "@parcel/packager-raw" "2.6.0" + "@parcel/reporter-dev-server" "2.6.0" + "@parcel/resolver-default" "2.6.0" + "@parcel/runtime-browser-hmr" "2.6.0" + "@parcel/runtime-js" "2.6.0" + "@parcel/runtime-react-refresh" "2.6.0" + "@parcel/runtime-service-worker" "2.6.0" + "@parcel/transformer-js" "2.6.0" + "@parcel/transformer-json" "2.6.0" + "@parcel/transformer-raw" "2.6.0" + "@parcel/transformer-react-refresh-wrap" "2.6.0" + +gatsby-plugin-gatsby-cloud@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-gatsby-cloud/-/gatsby-plugin-gatsby-cloud-4.17.0.tgz#2f00b2a119d8fcd477c450a569ad399db4cdf018" + integrity sha512-xO+CAtba+P6W+L1STv63D17iDYV9IOf1jQyXoDdxYd9CYy8g8VBMwFv1exVw+v9bGsPPRIJhFJM5wihw8W/hdg== + dependencies: + "@babel/runtime" "^7.15.4" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + gatsby-telemetry "^3.17.0" + kebab-hash "^0.1.2" + lodash "^4.17.21" + webpack-assets-manifest "^5.1.0" + +gatsby-plugin-image@^2.17.0: + version "2.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-image/-/gatsby-plugin-image-2.17.0.tgz#8106818cf09409f22356a61c7e1a2ceefc4f934b" + integrity sha512-YBVYczsxq04oA3+tx80f94FSxtCjFKc9tKj68Zp/NRZGmLL5/0/s++UcMHxzusmC2P/rAKfDlQ6HNOhm5dwFhQ== + dependencies: + "@babel/code-frame" "^7.14.0" + "@babel/parser" "^7.15.5" + "@babel/runtime" "^7.15.4" + "@babel/traverse" "^7.15.4" + babel-jsx-utils "^1.1.0" + babel-plugin-remove-graphql-queries "^4.17.0" + camelcase "^5.3.1" + chokidar "^3.5.2" + common-tags "^1.8.2" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + gatsby-plugin-utils "^3.11.0" + objectFitPolyfill "^2.3.5" + prop-types "^15.8.1" + +gatsby-plugin-manifest@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-manifest/-/gatsby-plugin-manifest-4.17.0.tgz#ac3d8be81629953f8af8e0570fdfc2fad02401db" + integrity sha512-LGxK2tko8DEGLVQEd4FRH3gTDdfSvhOXDuGJxtP5uz382DgVPbfYjU5KO72T6nZ8YJUOyOibCN8o+rw4OzvMzQ== + dependencies: + "@babel/runtime" "^7.15.4" + gatsby-core-utils "^3.17.0" + gatsby-plugin-utils "^3.11.0" + semver "^7.3.7" + sharp "^0.30.3" + +gatsby-plugin-mantine@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-mantine/-/gatsby-plugin-mantine-4.0.0.tgz#698de800cff065e73ee5565c99480ba73426e18b" + integrity sha512-7Cc0f01H7DEsuLhe+0Qst0oKlZB/jhxRCL4WMW7m+fSc3s57fBrL44ObhFlWCWQtLJkSsOx9Fl4Se4FN+gwySA== + dependencies: + "@mantine/ssr" ">=3.6.14" + +gatsby-plugin-offline@^5.17.0: + version "5.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-offline/-/gatsby-plugin-offline-5.17.0.tgz#e615c31b2522de5e9cade2a2f975e4c0310e113d" + integrity sha512-4sGpLkjOrv/b6E9FI6TqAX92D47PGbgCubqQTTV7c4zcEIjq9MA86VzNRV/sU8Jys+9Az0kBOIA8e7NEckyvXQ== + dependencies: + "@babel/runtime" "^7.15.4" + cheerio "^1.0.0-rc.10" + gatsby-core-utils "^3.17.0" + glob "^7.2.3" + idb-keyval "^3.2.0" + lodash "^4.17.21" + workbox-build "^4.3.1" + +gatsby-plugin-page-creator@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-page-creator/-/gatsby-plugin-page-creator-4.17.0.tgz#2a5521fddce5c626ee2fb2bee498c1ed77e64f68" + integrity sha512-Nv1lryCAfo5cWqkVnnqiMC6bmXXXOhdl2efLZKK88QwqT4tZO51gGrgsZGL4mtXtUUSkJXGE8pmjelUOcR+23g== + dependencies: + "@babel/runtime" "^7.15.4" + "@babel/traverse" "^7.15.4" + "@sindresorhus/slugify" "^1.1.2" + chokidar "^3.5.2" + fs-exists-cached "^1.0.0" + gatsby-core-utils "^3.17.0" + gatsby-page-utils "^2.17.0" + gatsby-plugin-utils "^3.11.0" + gatsby-telemetry "^3.17.0" + globby "^11.1.0" + lodash "^4.17.21" + +gatsby-plugin-postcss@^5.16.0: + version "5.16.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-postcss/-/gatsby-plugin-postcss-5.16.0.tgz#32cc2ab7b34fb90db2ef0918cbc4f34674a27259" + integrity sha512-72iZLQoYyJuRIZV+nHNYpP7PHTq5/hlEgzljk9mOhmtyxXMdvq3slw2YfK0ORpXOX+jxV7M2PwjybQYbfWCz2w== + dependencies: + "@babel/runtime" "^7.15.4" + postcss-loader "^4.3.0" + +gatsby-plugin-react-helmet@^5.17.0: + version "5.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-react-helmet/-/gatsby-plugin-react-helmet-5.17.0.tgz#0d092eb8771aadcf27d8a0f0b4363a9c449cc812" + integrity sha512-Lp52GFGNQO0Rt0VfYbFMFBw67XNaIITvwhBMqzxnIw/OCnnyjhylfYMpmyDQKT5S/yaWEF3MsvPmE2YCbXbSww== + dependencies: + "@babel/runtime" "^7.15.4" + +gatsby-plugin-sass@^5.14.0: + version "5.16.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-sass/-/gatsby-plugin-sass-5.16.0.tgz#a14b49e91a9de714e04dd094609f539b1170d5a3" + integrity sha512-iLko0j3JlGCmZiCmsXXzct8SFGGOtkZ1fhopBruSDTMuYUhAsLVS88IxP/o2TvjTbkpYgqPh3rBR+d4PLty5YA== + dependencies: + "@babel/runtime" "^7.15.4" + resolve-url-loader "^3.1.4" + sass-loader "^10.1.1" + +gatsby-plugin-sharp@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-sharp/-/gatsby-plugin-sharp-4.17.0.tgz#157901c4e3ac728f562d08d4a846fa1bd888a81c" + integrity sha512-+wP4hskMBSNgPYqBX2uqoqb5JUtMMvaZTzYNkZmnKkg85rHeDpUy+eMoCqrhh8wpW/NgcbQ9o6uK43jj2qv0xQ== + dependencies: + "@babel/runtime" "^7.15.4" + "@gatsbyjs/potrace" "^2.2.0" + async "^3.2.3" + bluebird "^3.7.2" + debug "^4.3.4" + filenamify "^4.3.0" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + gatsby-plugin-utils "^3.11.0" + gatsby-telemetry "^3.17.0" + got "^11.8.3" + lodash "^4.17.21" + mini-svg-data-uri "^1.4.4" + probe-image-size "^7.2.3" + progress "^2.0.3" + semver "^7.3.7" + sharp "^0.30.3" + svgo "1.3.2" + +gatsby-plugin-typescript@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-typescript/-/gatsby-plugin-typescript-4.17.0.tgz#95da5c9c4dbdd029eb702e465ccc4b10a12a8993" + integrity sha512-aSO/h4z+jdWBZtEnH17NoZaFchO8LkEcIdNX2saYIjJfa2tMbOR+wnOuQJVAdj61n8ntc1pQkNJxiEK31IaZjw== + dependencies: + "@babel/core" "^7.15.5" + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.14.5" + "@babel/plugin-proposal-numeric-separator" "^7.14.5" + "@babel/plugin-proposal-optional-chaining" "^7.14.5" + "@babel/preset-typescript" "^7.15.0" + "@babel/runtime" "^7.15.4" + babel-plugin-remove-graphql-queries "^4.17.0" + +gatsby-plugin-utils@^3.11.0: + version "3.11.0" + resolved "https://registry.npmmirror.com/gatsby-plugin-utils/-/gatsby-plugin-utils-3.11.0.tgz#cffbd98ab0cafe2ff9b8eb513c980024605dcadf" + integrity sha512-v2D6O2aME9p7tqlGykq5mgtj7Jqp/CWatgo0zJ7bFWSYPPUpQ7jRVAph1PIgFTAV/CaUrrvt9sxdRaZrglJyug== + dependencies: + "@babel/runtime" "^7.15.4" + "@gatsbyjs/potrace" "^2.2.0" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + gatsby-sharp "^0.11.0" + graphql-compose "^9.0.7" + import-from "^4.0.0" + joi "^17.4.2" + mime "^3.0.0" + mini-svg-data-uri "^1.4.4" + svgo "^2.8.0" + +gatsby-react-router-scroll@^5.17.0: + version "5.17.0" + resolved "https://registry.npmmirror.com/gatsby-react-router-scroll/-/gatsby-react-router-scroll-5.17.0.tgz#c145241d77d80a2c37e72c6e34c5061faf784228" + integrity sha512-RwFeZlKL0T8dxpEdlmuQAH+h4ugrQQpl4U2m0Wdo/F9/8/VEiAeX64oa2r2BqiiTf4MzAIIT+5/ONt83FcmZWg== + dependencies: + "@babel/runtime" "^7.15.4" + prop-types "^15.8.1" + +gatsby-remark-autolink-headers@^5.16.0: + version "5.16.0" + resolved "https://registry.npmmirror.com/gatsby-remark-autolink-headers/-/gatsby-remark-autolink-headers-5.16.0.tgz#88b3af3ca7caf9e8e82acd098d8fbf23c604f34e" + integrity sha512-IvbBBeX+CTAhoEGRhfUxzmZg+9fv+UvJTMArjJuB6cOQht1glwfPGoeAP1gm3kb3RDBYZulziTpsYZB5dE/TKQ== + dependencies: + "@babel/runtime" "^7.15.4" + github-slugger "^1.3.0" + lodash "^4.17.21" + mdast-util-to-string "^2.0.0" + unist-util-visit "^2.0.3" + +gatsby-remark-images@^6.17.0: + version "6.17.0" + resolved "https://registry.npmmirror.com/gatsby-remark-images/-/gatsby-remark-images-6.17.0.tgz#d666466f4674b29bf1bc7bb001e122a11ebf229e" + integrity sha512-pjUBNCfKT8mo1mXztHlxB5OYGSUUr8snTKvBOlKkDQUpQmyDp9BKTYcTMdhlrGqJEpkmMjFpXQSJPSxoTqw+7A== + dependencies: + "@babel/runtime" "^7.15.4" + "@gatsbyjs/potrace" "^2.2.0" + chalk "^4.1.2" + cheerio "^1.0.0-rc.10" + gatsby-core-utils "^3.17.0" + is-relative-url "^3.0.0" + lodash "^4.17.21" + mdast-util-definitions "^4.0.0" + query-string "^6.14.1" + unist-util-select "^3.0.4" + unist-util-visit-parents "^3.1.1" + +gatsby-remark-table-of-contents@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/gatsby-remark-table-of-contents/-/gatsby-remark-table-of-contents-2.0.0.tgz#9df384c5915a6b038deedcf831a1997fbb830974" + integrity sha512-Hl/zfvwX0FWMjaurzOTLpHNQ4rOd4AXnNrRsqJLwZPecZBc42UDX/JFL4hYS/NfrPd/svtwipkBXGo51CaKFXw== + dependencies: + js-yaml "^3.14.1" + mdast-util-toc "^5.1.0" + +gatsby-script@^1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/gatsby-script/-/gatsby-script-1.2.0.tgz#1655b14c2eb9cda731e03272dfae8c6a709ba785" + integrity sha512-Fkf6yJxwuQCS6xdFxF0gpiCUiS3PYnRv5I2mCAJV0Cu7k4PYgVpQJt79bcDHvnx755VpXOmZIfMXegACg1XS0w== + +gatsby-sharp@^0.11.0: + version "0.11.0" + resolved "https://registry.npmmirror.com/gatsby-sharp/-/gatsby-sharp-0.11.0.tgz#f672e26a4134e2ff264a335ab6efd35782de8052" + integrity sha512-RIbU8qi/Qs7G/KJiY0zyGS9Yic5n4RpDUf/1W3gvRl78Uo3LSuIeGEpaai6mYEnreuGb5fQIcqdkYs/UV3e8WA== + dependencies: + "@types/sharp" "^0.30.0" + sharp "^0.30.3" + +gatsby-source-filesystem@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-source-filesystem/-/gatsby-source-filesystem-4.17.0.tgz#5cbb8615151aa5ac167aea328be340453cfca0a9" + integrity sha512-1UErm5fcAqHOB3439V2BmSB5Kz+yDN15qj5J503/f6nL44kPPhJKSjjR64A5iz2y/3nCSTyZfmSs44jw9I1hhg== + dependencies: + "@babel/runtime" "^7.15.4" + chokidar "^3.5.2" + file-type "^16.5.3" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + got "^9.6.0" + md5-file "^5.0.0" + mime "^2.5.2" + pretty-bytes "^5.4.1" + progress "^2.0.3" + valid-url "^1.0.9" + xstate "^4.26.1" + +gatsby-telemetry@^3.17.0: + version "3.17.0" + resolved "https://registry.npmmirror.com/gatsby-telemetry/-/gatsby-telemetry-3.17.0.tgz#de5b094340d714969ac5a5f01ed19999d3b9557e" + integrity sha512-1N4uKj2Yj7AykLmwjsdOmNirf3UmP8CBXjj/LmWf5cLDVbdPqkirAlyD6uTeEliG5UFHPTlBsRi26v558mFYJQ== + dependencies: + "@babel/code-frame" "^7.14.0" + "@babel/runtime" "^7.15.4" + "@turist/fetch" "^7.1.7" + "@turist/time" "^0.0.2" + async-retry-ng "^2.0.1" + boxen "^4.2.0" + configstore "^5.0.1" + fs-extra "^10.1.0" + gatsby-core-utils "^3.17.0" + git-up "^4.0.5" + is-docker "^2.2.1" + lodash "^4.17.21" + node-fetch "^2.6.7" + +gatsby-transformer-json@^4.15.0: + version "4.16.0" + resolved "https://registry.npmmirror.com/gatsby-transformer-json/-/gatsby-transformer-json-4.16.0.tgz#9d905367d009683897887b0319d51473869c12f8" + integrity sha512-Z/IiLGtoVdwI6di/HeTau9RdKfRbdGotPOaXvyRP0hx2TmmZUKtcJr4q57EC4EYVlDaKMxOqKjEeMP/pDHixDg== + dependencies: + "@babel/runtime" "^7.15.4" + bluebird "^3.7.2" + +gatsby-transformer-plaintext@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/gatsby-transformer-plaintext/-/gatsby-transformer-plaintext-2.0.0.tgz#0390c7596c4b129c08f8990871cac57a0e2c8739" + integrity sha512-mrv9UPsDyoYzVBwWq7toL32nQeC9+HgANOhEvosnD+o89DnrfEfmeScQ5mYhxD17hgR13HqM/PO+aru0xD0W6g== + dependencies: + "@babel/runtime" "^7.13.10" + +gatsby-transformer-remark@^5.14.0: + version "5.16.0" + resolved "https://registry.npmmirror.com/gatsby-transformer-remark/-/gatsby-transformer-remark-5.16.0.tgz#647f704e97cd4e38e342244f60c504a065134e1d" + integrity sha512-hJO7VPWwDq71QsSQ28GafVVczmMQm3Qt0oydzKdcSbF4b2tc8+QTQGn3EhPDDUt+2mDBat2hNohS8EvNaiEpEA== + dependencies: + "@babel/runtime" "^7.15.4" + gatsby-core-utils "^3.16.0" + gray-matter "^4.0.3" + hast-util-raw "^6.0.2" + hast-util-to-html "^7.1.3" + lodash "^4.17.21" + mdast-util-to-hast "^10.2.0" + mdast-util-to-string "^2.0.0" + mdast-util-toc "^5.1.0" + remark "^13.0.0" + remark-footnotes "^3.0.0" + remark-gfm "^1.0.0" + remark-parse "^9.0.0" + remark-retext "^4.0.0" + remark-stringify "^9.0.1" + retext-english "^3.0.4" + sanitize-html "^1.27.5" + underscore.string "^3.3.6" + unified "^9.2.2" + unist-util-remove-position "^3.0.0" + unist-util-select "^3.0.4" + unist-util-visit "^2.0.3" + +gatsby-transformer-sharp@^4.17.0: + version "4.17.0" + resolved "https://registry.npmmirror.com/gatsby-transformer-sharp/-/gatsby-transformer-sharp-4.17.0.tgz#abc2eddd4b28d268eeca6363d22b3e416a33ac53" + integrity sha512-yaBzgqpjMqpy1ZuJ4CRbYh8rdiyvuSlMS0aMRiGlcjgUmKuT5qGJsQWsuhihbKWqn8OPhNzrAsnWvWnOVjAeCg== + dependencies: + "@babel/runtime" "^7.15.4" + "@gatsbyjs/potrace" "^2.2.0" + bluebird "^3.7.2" + common-tags "^1.8.2" + fs-extra "^10.1.0" + gatsby-plugin-utils "^3.11.0" + probe-image-size "^7.2.3" + semver "^7.3.7" + sharp "^0.30.3" + +gatsby-worker@^1.17.0: + version "1.17.0" + resolved "https://registry.npmmirror.com/gatsby-worker/-/gatsby-worker-1.17.0.tgz#ef6616023e94c8bfc504f7853c210027e2ee449e" + integrity sha512-hA7awNHmcNWp3TgQkjF0Z3I9YGWpdsf8jUdJOlAjI+aBxQ2vdHTVNwbMdXwtlf9umQRwORnmEzADmQdm0Cx5AA== + dependencies: + "@babel/core" "^7.15.5" + "@babel/runtime" "^7.15.4" + +gatsby@^4.17.2: + version "4.17.2" + resolved "https://registry.npmmirror.com/gatsby/-/gatsby-4.17.2.tgz#0ab2b8abf9e0601161732f2a0c60042e6c61dfb0" + integrity sha512-uMNH/X/Fd0kGjelgAPTukgQhxu/aMhRj5YJ07SM4D2yBuO+c4+y1dZRz5q9EUryCeVSx1cSt0J6JMkfzOXmbyA== + dependencies: + "@babel/code-frame" "^7.14.0" + "@babel/core" "^7.15.5" + "@babel/eslint-parser" "^7.15.4" + "@babel/helper-plugin-utils" "^7.14.5" + "@babel/parser" "^7.15.5" + "@babel/runtime" "^7.15.4" + "@babel/traverse" "^7.15.4" + "@babel/types" "^7.15.4" + "@builder.io/partytown" "^0.5.2" + "@gatsbyjs/reach-router" "^1.3.6" + "@gatsbyjs/webpack-hot-middleware" "^2.25.2" + "@graphql-codegen/add" "^3.1.1" + "@graphql-codegen/core" "^2.5.1" + "@graphql-codegen/plugin-helpers" "^2.4.2" + "@graphql-codegen/typescript" "^2.4.8" + "@graphql-codegen/typescript-operations" "^2.3.5" + "@graphql-tools/code-file-loader" "^7.2.14" + "@graphql-tools/load" "^7.5.10" + "@jridgewell/trace-mapping" "^0.3.13" + "@nodelib/fs.walk" "^1.2.8" + "@parcel/core" "2.6.0" + "@pmmmwh/react-refresh-webpack-plugin" "^0.4.3" + "@types/http-proxy" "^1.17.7" + "@typescript-eslint/eslint-plugin" "^4.33.0" + "@typescript-eslint/parser" "^4.33.0" + "@vercel/webpack-asset-relocator-loader" "^1.7.0" + address "1.1.2" + anser "^2.1.0" + autoprefixer "^10.4.0" + axios "^0.21.1" + babel-loader "^8.2.3" + babel-plugin-add-module-exports "^1.0.4" + babel-plugin-dynamic-import-node "^2.3.3" + babel-plugin-lodash "^3.3.4" + babel-plugin-remove-graphql-queries "^4.17.0" + babel-preset-gatsby "^2.17.0" + better-opn "^2.1.1" + bluebird "^3.7.2" + body-parser "^1.19.0" + browserslist "^4.17.5" + cache-manager "^2.11.1" + chalk "^4.1.2" + chokidar "^3.5.2" + common-tags "^1.8.0" + compression "^1.7.4" + cookie "^0.4.1" + core-js "^3.22.3" + cors "^2.8.5" + css-loader "^5.2.7" + css-minimizer-webpack-plugin "^2.0.0" + css.escape "^1.5.1" + date-fns "^2.25.0" + debug "^3.2.7" + deepmerge "^4.2.2" + detect-port "^1.3.0" + devcert "^1.2.0" + dotenv "^8.6.0" + enhanced-resolve "^5.8.3" + eslint "^7.32.0" + eslint-config-react-app "^6.0.0" + eslint-plugin-flowtype "^5.10.0" + eslint-plugin-graphql "^4.0.0" + eslint-plugin-import "^2.26.0" + eslint-plugin-jsx-a11y "^6.5.1" + eslint-plugin-react "^7.29.4" + eslint-plugin-react-hooks "^4.5.0" + eslint-webpack-plugin "^2.6.0" + event-source-polyfill "1.0.25" + execa "^5.1.1" + express "^4.17.1" + express-graphql "^0.12.0" + express-http-proxy "^1.6.3" + fastest-levenshtein "^1.0.12" + fastq "^1.13.0" + file-loader "^6.2.0" + find-cache-dir "^3.3.2" + fs-exists-cached "1.0.0" + fs-extra "^10.1.0" + gatsby-cli "^4.17.1" + gatsby-core-utils "^3.17.0" + gatsby-graphiql-explorer "^2.17.0" + gatsby-legacy-polyfills "^2.17.0" + gatsby-link "^4.17.0" + gatsby-page-utils "^2.17.0" + gatsby-parcel-config "0.8.0" + gatsby-plugin-page-creator "^4.17.0" + gatsby-plugin-typescript "^4.17.0" + gatsby-plugin-utils "^3.11.0" + gatsby-react-router-scroll "^5.17.0" + gatsby-script "^1.2.0" + gatsby-telemetry "^3.17.0" + gatsby-worker "^1.17.0" + glob "^7.2.3" + globby "^11.1.0" + got "^11.8.2" + graphql "^15.7.2" + graphql-compose "^9.0.7" + graphql-playground-middleware-express "^1.7.22" + hasha "^5.2.2" + http-proxy "^1.18.1" + invariant "^2.2.4" + is-relative "^1.0.0" + is-relative-url "^3.0.0" + joi "^17.4.2" + json-loader "^0.5.7" + latest-version "5.1.0" + lmdb "2.5.2" + lodash "^4.17.21" + md5-file "^5.0.0" + meant "^1.0.3" + memoizee "^0.4.15" + micromatch "^4.0.4" + mime "^2.5.2" + mini-css-extract-plugin "1.6.2" + mitt "^1.2.0" + moment "^2.29.1" + multer "^1.4.3" + node-fetch "^2.6.6" + normalize-path "^3.0.0" + null-loader "^4.0.1" + opentracing "^0.14.5" + p-defer "^3.0.0" + parseurl "^1.3.3" + physical-cpu-count "^2.0.0" + platform "^1.3.6" + postcss "^8.3.11" + postcss-flexbugs-fixes "^5.0.2" + postcss-loader "^5.3.0" + prompts "^2.4.2" + prop-types "^15.7.2" + query-string "^6.14.1" + raw-loader "^4.0.2" + react-dev-utils "^12.0.1" + react-refresh "^0.9.0" + redux "4.1.2" + redux-thunk "^2.4.0" + resolve-from "^5.0.0" + semver "^7.3.7" + shallow-compare "^1.2.2" + signal-exit "^3.0.5" + slugify "^1.6.1" + socket.io "3.1.2" + socket.io-client "3.1.3" + source-map-support "^0.5.20" + st "^2.0.0" + stack-trace "^0.0.10" + string-similarity "^1.2.2" + strip-ansi "^6.0.1" + style-loader "^2.0.0" + terser-webpack-plugin "^5.2.4" + tmp "^0.2.1" + "true-case-path" "^2.2.1" + type-of "^2.0.1" + url-loader "^4.1.1" + uuid "^8.3.2" + webpack "^5.61.0" + webpack-dev-middleware "^4.3.0" + webpack-merge "^5.8.0" + webpack-stats-plugin "^1.0.3" + webpack-virtual-modules "^0.3.2" + xstate "^4.26.0" + yaml-loader "^0.6.0" + optionalDependencies: + gatsby-sharp "^0.11.0" + +gauge@~2.7.3: + version "2.7.4" + resolved "https://registry.npmmirror.com/gauge/-/gauge-2.7.4.tgz#2c03405c7538c39d7eb37b317022e325fb018bf7" + integrity sha512-14x4kjc6lkD3ltw589k0NrPD6cCNTD6CWoVUNpB85+DrtONoZn+Rug6xZU5RvSC4+TZPxA5AnBibQYAvZn41Hg== + dependencies: + aproba "^1.0.3" + console-control-strings "^1.0.0" + has-unicode "^2.0.0" + object-assign "^4.1.0" + signal-exit "^3.0.0" + string-width "^1.0.1" + strip-ansi "^3.0.1" + wide-align "^1.1.0" + +gaze@^1.0.0: + version "1.1.3" + resolved "https://registry.npmmirror.com/gaze/-/gaze-1.1.3.tgz#c441733e13b927ac8c0ff0b4c3b033f28812924a" + integrity sha512-BRdNm8hbWzFzWHERTrejLqwHDfS4GibPoq5wjTPIoJHoBtKGPg3xAFfxmM+9ztbXelxcf2hwQcaz1PtmFeue8g== + dependencies: + globule "^1.0.0" + +gensync@^1.0.0-beta.2: + version "1.0.0-beta.2" + resolved "https://registry.npmmirror.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" + integrity sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg== + +get-caller-file@^2.0.1: + version "2.0.5" + resolved "https://registry.npmmirror.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" + integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== + +get-intrinsic@^1.0.2, get-intrinsic@^1.1.0, get-intrinsic@^1.1.1: + version "1.1.2" + resolved "https://registry.npmmirror.com/get-intrinsic/-/get-intrinsic-1.1.2.tgz#336975123e05ad0b7ba41f152ee4aadbea6cf598" + integrity sha512-Jfm3OyCxHh9DJyc28qGk+JmfkpO41A4XkneDSujN9MDXrm4oDKdHvndhZ2dN94+ERNfkYJWDclW6k2L/ZGHjXA== + dependencies: + function-bind "^1.1.1" + has "^1.0.3" + has-symbols "^1.0.3" + +get-own-enumerable-property-symbols@^3.0.0: + version "3.0.2" + resolved "https://registry.npmmirror.com/get-own-enumerable-property-symbols/-/get-own-enumerable-property-symbols-3.0.2.tgz#b5fde77f22cbe35f390b4e089922c50bce6ef664" + integrity sha512-I0UBV/XOz1XkIJHEUDMZAbzCThU/H8DxmSfmdGcKPnVhu2VfFqr34jr9777IyaTYvxjedWhqVIilEDsCdP5G6g== + +get-port@^3.2.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/get-port/-/get-port-3.2.0.tgz#dd7ce7de187c06c8bf353796ac71e099f0980ebc" + integrity sha512-x5UJKlgeUiNT8nyo/AcnwLnZuZNcSjSw0kogRB+Whd1fjjFq4B1hySFxSFWWSn4mIBzg3sRNUDFYc4g5gjPoLg== + +get-stdin@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/get-stdin/-/get-stdin-4.0.1.tgz#b968c6b0a04384324902e8bf1a5df32579a450fe" + integrity sha512-F5aQMywwJ2n85s4hJPTT9RPxGmubonuB10MNYo17/xph174n2MIR33HRguhzVag10O/npM7SPk73LMZNP+FaWw== + +get-stream@^4.0.0, get-stream@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/get-stream/-/get-stream-4.1.0.tgz#c1b255575f3dc21d59bfc79cd3d2b46b1c3a54b5" + integrity sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w== + dependencies: + pump "^3.0.0" + +get-stream@^5.1.0: + version "5.2.0" + resolved "https://registry.npmmirror.com/get-stream/-/get-stream-5.2.0.tgz#4966a1795ee5ace65e706c4b7beb71257d6e22d3" + integrity sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA== + dependencies: + pump "^3.0.0" + +get-stream@^6.0.0: + version "6.0.1" + resolved "https://registry.npmmirror.com/get-stream/-/get-stream-6.0.1.tgz#a262d8eef67aced57c2852ad6167526a43cbf7b7" + integrity sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg== + +get-symbol-description@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/get-symbol-description/-/get-symbol-description-1.0.0.tgz#7fdb81c900101fbd564dd5f1a30af5aadc1e58d6" + integrity sha512-2EmdH1YvIQiZpltCNgkuiUnyukzxM/R6NDJX31Ke3BG1Nq5b0S2PhX59UKi9vZpPDQVdqn+1IcaAwnzTT5vCjw== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.1.1" + +getpass@^0.1.1: + version "0.1.7" + resolved "https://registry.npmmirror.com/getpass/-/getpass-0.1.7.tgz#5eff8e3e684d569ae4cb2b1282604e8ba62149fa" + integrity sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng== + dependencies: + assert-plus "^1.0.0" + +gifwrap@^0.9.2: + version "0.9.4" + resolved "https://registry.npmmirror.com/gifwrap/-/gifwrap-0.9.4.tgz#f4eb6169ba027d61df64aafbdcb1f8ae58ccc0c5" + integrity sha512-MDMwbhASQuVeD4JKd1fKgNgCRL3fGqMM4WaqpNhWO0JiMOAjbQdumbs4BbBZEy9/M00EHEjKN3HieVhCUlwjeQ== + dependencies: + image-q "^4.0.0" + omggif "^1.0.10" + +git-up@^4.0.5: + version "4.0.5" + resolved "https://registry.npmmirror.com/git-up/-/git-up-4.0.5.tgz#e7bb70981a37ea2fb8fe049669800a1f9a01d759" + integrity sha512-YUvVDg/vX3d0syBsk/CKUTib0srcQME0JyHkL5BaYdwLsiCslPWmDSi8PUMo9pXYjrryMcmsCoCgsTpSCJEQaA== + dependencies: + is-ssh "^1.3.0" + parse-url "^6.0.0" + +github-from-package@0.0.0: + version "0.0.0" + resolved "https://registry.npmmirror.com/github-from-package/-/github-from-package-0.0.0.tgz#97fb5d96bfde8973313f20e8288ef9a167fa64ce" + integrity sha512-SyHy3T1v2NUXn29OsWdxmK6RwHD+vkj3v8en8AOBZ1wBQ/hCAQ5bAQTD02kW4W9tUp/3Qh6J8r9EvntiyCmOOw== + +github-slugger@^1.2.1, github-slugger@^1.3.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/github-slugger/-/github-slugger-1.4.0.tgz#206eb96cdb22ee56fdc53a28d5a302338463444e" + integrity sha512-w0dzqw/nt51xMVmlaV1+JRzN+oCa1KfcgGEWhxUG16wbdA+Xnt/yoFO8Z8x/V82ZcZ0wy6ln9QDup5avbhiDhQ== + +glob-parent@^5.1.2, glob-parent@~5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/glob-parent/-/glob-parent-5.1.2.tgz#869832c58034fe68a4093c17dc15e8340d8401c4" + integrity sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow== + dependencies: + is-glob "^4.0.1" + +glob-parent@^6.0.2: + version "6.0.2" + resolved "https://registry.npmmirror.com/glob-parent/-/glob-parent-6.0.2.tgz#6d237d99083950c79290f24c7642a3de9a28f9e3" + integrity sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A== + dependencies: + is-glob "^4.0.3" + +glob-to-regexp@^0.4.1: + version "0.4.1" + resolved "https://registry.npmmirror.com/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz#c75297087c851b9a578bd217dd59a92f59fe546e" + integrity sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw== + +glob@^7.0.0, 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.2.3: + version "7.2.3" + resolved "https://registry.npmmirror.com/glob/-/glob-7.2.3.tgz#b8df0fb802bbfa8e89bd1d938b4e16578ed44f2b" + integrity sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.1.1" + once "^1.3.0" + path-is-absolute "^1.0.0" + +glob@~7.1.1: + version "7.1.7" + resolved "https://registry.npmmirror.com/glob/-/glob-7.1.7.tgz#3b193e9233f01d42d0b3f78294bbeeb418f94a90" + integrity sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.0.4" + once "^1.3.0" + path-is-absolute "^1.0.0" + +global-dirs@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/global-dirs/-/global-dirs-3.0.0.tgz#70a76fe84ea315ab37b1f5576cbde7d48ef72686" + integrity sha512-v8ho2DS5RiCjftj1nD9NmnfaOzTdud7RRnVd9kFNOjqZbISlx5DQ+OrTkywgd0dIt7oFCvKetZSHoHcP3sDdiA== + dependencies: + ini "2.0.0" + +global-modules@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/global-modules/-/global-modules-2.0.0.tgz#997605ad2345f27f51539bea26574421215c7780" + integrity sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A== + dependencies: + global-prefix "^3.0.0" + +global-prefix@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/global-prefix/-/global-prefix-3.0.0.tgz#fc85f73064df69f50421f47f883fe5b913ba9b97" + integrity sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg== + dependencies: + ini "^1.3.5" + kind-of "^6.0.2" + which "^1.3.1" + +global@~4.4.0: + version "4.4.0" + resolved "https://registry.npmmirror.com/global/-/global-4.4.0.tgz#3e7b105179006a323ed71aafca3e9c57a5cc6406" + integrity sha512-wv/LAoHdRE3BeTGz53FAamhGlPLhlssK45usmGFThIi4XqnBmjKQ16u+RNbP7WvigRZDxUsM0J3gcQ5yicaL0w== + dependencies: + min-document "^2.19.0" + process "^0.11.10" + +globals@^11.1.0: + version "11.12.0" + resolved "https://registry.npmmirror.com/globals/-/globals-11.12.0.tgz#ab8795338868a0babd8525758018c2a7eb95c42e" + integrity sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA== + +globals@^13.2.0, globals@^13.6.0, globals@^13.9.0: + version "13.15.0" + resolved "https://registry.npmmirror.com/globals/-/globals-13.15.0.tgz#38113218c907d2f7e98658af246cef8b77e90bac" + integrity sha512-bpzcOlgDhMG070Av0Vy5Owklpv1I6+j96GhUI7Rh7IzDCKLzboflLrrfqMu8NquDbiR4EOQk7XzJwqVJxicxog== + dependencies: + type-fest "^0.20.2" + +globby@11.0.3: + version "11.0.3" + resolved "https://registry.npmmirror.com/globby/-/globby-11.0.3.tgz#9b1f0cb523e171dd1ad8c7b2a9fb4b644b9593cb" + integrity sha512-ffdmosjA807y7+lA1NM0jELARVmYul/715xiILEjo3hBLPTcirgQNnXECn5g3mtR8TOLCVbkfua1Hpen25/Xcg== + dependencies: + array-union "^2.1.0" + dir-glob "^3.0.1" + fast-glob "^3.1.1" + ignore "^5.1.4" + merge2 "^1.3.0" + slash "^3.0.0" + +globby@^11.0.3, globby@^11.0.4, globby@^11.1.0: + version "11.1.0" + resolved "https://registry.npmmirror.com/globby/-/globby-11.1.0.tgz#bd4be98bb042f83d796f7e3811991fbe82a0d34b" + integrity sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g== + dependencies: + array-union "^2.1.0" + dir-glob "^3.0.1" + fast-glob "^3.2.9" + ignore "^5.2.0" + merge2 "^1.4.1" + slash "^3.0.0" + +globule@^1.0.0: + version "1.3.3" + resolved "https://registry.npmmirror.com/globule/-/globule-1.3.3.tgz#811919eeac1ab7344e905f2e3be80a13447973c2" + integrity sha512-mb1aYtDbIjTu4ShMB85m3UzjX9BVKe9WCzsnfMSZk+K5GpIbBOexgg4PPCt5eHDEG5/ZQAUX2Kct02zfiPLsKg== + dependencies: + glob "~7.1.1" + lodash "~4.17.10" + minimatch "~3.0.2" + +got@^11.8.2, got@^11.8.3: + version "11.8.5" + resolved "https://registry.npmmirror.com/got/-/got-11.8.5.tgz#ce77d045136de56e8f024bebb82ea349bc730046" + integrity sha512-o0Je4NvQObAuZPHLFoRSkdG2lTgtcynqymzg2Vupdx6PorhaT5MCbIyXG6d4D94kk8ZG57QeosgdiqfJWhEhlQ== + dependencies: + "@sindresorhus/is" "^4.0.0" + "@szmarczak/http-timer" "^4.0.5" + "@types/cacheable-request" "^6.0.1" + "@types/responselike" "^1.0.0" + cacheable-lookup "^5.0.3" + cacheable-request "^7.0.2" + decompress-response "^6.0.0" + http2-wrapper "^1.0.0-beta.5.2" + lowercase-keys "^2.0.0" + p-cancelable "^2.0.0" + responselike "^2.0.0" + +got@^9.6.0: + version "9.6.0" + resolved "https://registry.npmmirror.com/got/-/got-9.6.0.tgz#edf45e7d67f99545705de1f7bbeeeb121765ed85" + integrity sha512-R7eWptXuGYxwijs0eV+v3o6+XH1IqVK8dJOEecQfTmkncw9AV4dcw/Dhxi8MdlqPthxxpZyizMzyg8RTmEsG+Q== + dependencies: + "@sindresorhus/is" "^0.14.0" + "@szmarczak/http-timer" "^1.1.2" + cacheable-request "^6.0.0" + decompress-response "^3.3.0" + duplexer3 "^0.1.4" + get-stream "^4.1.0" + lowercase-keys "^1.0.1" + mimic-response "^1.0.1" + p-cancelable "^1.0.0" + to-readable-stream "^1.0.0" + url-parse-lax "^3.0.0" + +graceful-fs@^4.1.2, graceful-fs@^4.1.6, graceful-fs@^4.2.0, graceful-fs@^4.2.3, graceful-fs@^4.2.4, graceful-fs@^4.2.9: + version "4.2.10" + resolved "https://registry.npmmirror.com/graceful-fs/-/graceful-fs-4.2.10.tgz#147d3a006da4ca3ce14728c7aefc287c367d7a6c" + integrity sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA== + +graphql-compose@^9.0.7: + version "9.0.8" + resolved "https://registry.npmmirror.com/graphql-compose/-/graphql-compose-9.0.8.tgz#920b6b0584f5e3784a532138974063d3807f7455" + integrity sha512-I3zvygpVz5hOWk2cYL6yhbgfKbNWbiZFNXlWkv/55U+lX6Y3tL+SyY3zunw7QWrN/qtwG2DqZb13SHTv2MgdEQ== + dependencies: + graphql-type-json "0.3.2" + +graphql-config@^3.0.2: + version "3.4.1" + resolved "https://registry.npmmirror.com/graphql-config/-/graphql-config-3.4.1.tgz#59f937a1b4d3a3c2dcdb27ddf5b4d4d4b2c6e9e1" + integrity sha512-g9WyK4JZl1Ko++FSyE5Ir2g66njfxGzrDDhBOwnkoWf/t3TnnZG6BBkWP+pkqVJ5pqMJGPKHNrbew8jRxStjhw== + dependencies: + "@endemolshinegroup/cosmiconfig-typescript-loader" "3.0.2" + "@graphql-tools/graphql-file-loader" "^6.0.0" + "@graphql-tools/json-file-loader" "^6.0.0" + "@graphql-tools/load" "^6.0.0" + "@graphql-tools/merge" "6.0.0 - 6.2.14" + "@graphql-tools/url-loader" "^6.0.0" + "@graphql-tools/utils" "^7.0.0" + cosmiconfig "7.0.0" + cosmiconfig-toml-loader "1.0.0" + minimatch "3.0.4" + string-env-interpolation "1.0.1" + +graphql-playground-html@^1.6.30: + version "1.6.30" + resolved "https://registry.npmmirror.com/graphql-playground-html/-/graphql-playground-html-1.6.30.tgz#14c2a8eb7fc17bfeb1a746bbb28a11e34bf0b391" + integrity sha512-tpCujhsJMva4aqE8ULnF7/l3xw4sNRZcSHu+R00VV+W0mfp+Q20Plvcrp+5UXD+2yS6oyCXncA+zoQJQqhGCEw== + dependencies: + xss "^1.0.6" + +graphql-playground-middleware-express@^1.7.22: + version "1.7.23" + resolved "https://registry.npmmirror.com/graphql-playground-middleware-express/-/graphql-playground-middleware-express-1.7.23.tgz#95aba44d801ff3c08b2246917d2901d2e7c35d3d" + integrity sha512-M/zbTyC1rkgiQjFSgmzAv6umMHOphYLNWZp6Ye5QrD77WfGOOoSqDsVmGUczc2pDkEPEzzGB/bvBO5rdzaTRgw== + dependencies: + graphql-playground-html "^1.6.30" + +graphql-tag@^2.11.0: + version "2.12.6" + resolved "https://registry.npmmirror.com/graphql-tag/-/graphql-tag-2.12.6.tgz#d441a569c1d2537ef10ca3d1633b48725329b5f1" + integrity sha512-FdSNcu2QQcWnM2VNvSCCDCVS5PpPqpzgFT8+GXzqJuoDd0CBncxCY278u4mhRO7tMgo2JjgJA5aZ+nWSQ/Z+xg== + dependencies: + tslib "^2.1.0" + +graphql-type-json@0.3.2: + version "0.3.2" + resolved "https://registry.npmmirror.com/graphql-type-json/-/graphql-type-json-0.3.2.tgz#f53a851dbfe07bd1c8157d24150064baab41e115" + integrity sha512-J+vjof74oMlCWXSvt0DOf2APEdZOCdubEvGDUAlqH//VBYcOYsGgRW7Xzorr44LvkjiuvecWc8fChxuZZbChtg== + +graphql-ws@^4.4.1: + version "4.9.0" + resolved "https://registry.npmmirror.com/graphql-ws/-/graphql-ws-4.9.0.tgz#5cfd8bb490b35e86583d8322f5d5d099c26e365c" + integrity sha512-sHkK9+lUm20/BGawNEWNtVAeJzhZeBg21VmvmLoT5NdGVeZWv5PdIhkcayQIAgjSyyQ17WMKmbDijIPG2On+Ag== + +graphql@^15.7.2: + version "15.8.0" + resolved "https://registry.npmmirror.com/graphql/-/graphql-15.8.0.tgz#33410e96b012fa3bdb1091cc99a94769db212b38" + integrity sha512-5gghUc24tP9HRznNpV2+FIoq3xKkj5dTQqf4v0CpdPbFVwFkWoxOM+o+2OC9ZSvjEMTjfmG9QT+gcvggTwW1zw== + +gray-matter@^4.0.3: + version "4.0.3" + resolved "https://registry.npmmirror.com/gray-matter/-/gray-matter-4.0.3.tgz#e893c064825de73ea1f5f7d88c7a9f7274288798" + integrity sha512-5v6yZd4JK3eMI3FqqCouswVqwugaA9r4dNZB1wwcmrD02QkV5H0y7XBQW8QwQqEaZY1pM9aqORSORhJRdNK44Q== + dependencies: + js-yaml "^3.13.1" + kind-of "^6.0.2" + section-matter "^1.0.0" + strip-bom-string "^1.0.0" + +gzip-size@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/gzip-size/-/gzip-size-6.0.0.tgz#065367fd50c239c0671cbcbad5be3e2eeb10e462" + integrity sha512-ax7ZYomf6jqPTQ4+XCpUGyXKHk5WweS+e05MBO4/y3WJ5RkmPXNKvX+bx1behVILVwr6JSQvZAku021CHPXG3Q== + dependencies: + duplexer "^0.1.2" + +har-schema@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/har-schema/-/har-schema-2.0.0.tgz#a94c2224ebcac04782a0d9035521f24735b7ec92" + integrity sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q== + +har-validator@~5.1.3: + version "5.1.5" + resolved "https://registry.npmmirror.com/har-validator/-/har-validator-5.1.5.tgz#1f0803b9f8cb20c0fa13822df1ecddb36bde1efd" + integrity sha512-nmT2T0lljbxdQZfspsno9hgrG3Uir6Ks5afism62poxqBM6sDnMEuPmzTq8XN0OEwqKLLdh1jQI3qyE66Nzb3w== + dependencies: + ajv "^6.12.3" + har-schema "^2.0.0" + +hard-rejection@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/hard-rejection/-/hard-rejection-2.1.0.tgz#1c6eda5c1685c63942766d79bb40ae773cecd883" + integrity sha512-VIZB+ibDhx7ObhAe7OVtoEbuP4h/MuOTHJ+J8h/eBXotJYl0fBgR72xDFCKgIh22OJZIOVNxBMWuhAr10r8HdA== + +has-ansi@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/has-ansi/-/has-ansi-2.0.0.tgz#34f5049ce1ecdf2b0649af3ef24e45ed35416d91" + integrity sha512-C8vBJ8DwUCx19vhm7urhTuUsr4/IyP6l4VzNQDv+ryHQObW3TTTp9yB68WpYgRe2bbaGuZ/se74IqFeVnMnLZg== + dependencies: + ansi-regex "^2.0.0" + +has-bigints@^1.0.1, has-bigints@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/has-bigints/-/has-bigints-1.0.2.tgz#0871bd3e3d51626f6ca0966668ba35d5602d6eaa" + integrity sha512-tSvCKtBr9lkF0Ex0aQiP9N+OpV4zi2r/Nee5VkRDbaqv35RLYMzbwQfFSZZH0kR+Rd6302UJZ2p/bJCEoR3VoQ== + +has-cors@1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/has-cors/-/has-cors-1.1.0.tgz#5e474793f7ea9843d1bb99c23eef49ff126fff39" + integrity sha512-g5VNKdkFuUuVCP9gYfDJHjK2nqdQJ7aDLTnycnc2+RvsOQbuLdF5pm7vuE5J76SEBIQjs4kQY/BWq74JUmjbXA== + +has-flag@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/has-flag/-/has-flag-3.0.0.tgz#b5d454dc2199ae225699f3467e5a07f3b955bafd" + integrity sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw== + +has-flag@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b" + integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ== + +has-property-descriptors@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz#610708600606d36961ed04c196193b6a607fa861" + integrity sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ== + dependencies: + get-intrinsic "^1.1.1" + +has-symbols@^1.0.1, has-symbols@^1.0.2, has-symbols@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/has-symbols/-/has-symbols-1.0.3.tgz#bb7b2c4349251dce87b125f7bdf874aa7c8b39f8" + integrity sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A== + +has-tostringtag@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/has-tostringtag/-/has-tostringtag-1.0.0.tgz#7e133818a7d394734f941e73c3d3f9291e658b25" + integrity sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ== + dependencies: + has-symbols "^1.0.2" + +has-unicode@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/has-unicode/-/has-unicode-2.0.1.tgz#e0e6fe6a28cf51138855e086d1691e771de2a8b9" + integrity sha512-8Rf9Y83NBReMnx0gFzA8JImQACstCYWUplepDa9xprwwtmgEZUF0h/i5xSA625zB/I37EtrswSST6OXxwaaIJQ== + +has-yarn@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/has-yarn/-/has-yarn-2.1.0.tgz#137e11354a7b5bf11aa5cb649cf0c6f3ff2b2e77" + integrity sha512-UqBRqi4ju7T+TqGNdqAO0PaSVGsDGJUBQvk9eUWNGRY1CFGDzYhLWoM7JQEemnlvVcv/YEmc2wNW8BC24EnUsw== + +has@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/has/-/has-1.0.3.tgz#722d7cbfc1f6aa8241f16dd814e011e1f41e8796" + integrity sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw== + dependencies: + function-bind "^1.1.1" + +hasha@^5.2.2: + version "5.2.2" + resolved "https://registry.npmmirror.com/hasha/-/hasha-5.2.2.tgz#a48477989b3b327aea3c04f53096d816d97522a1" + integrity sha512-Hrp5vIK/xr5SkeN2onO32H0MgNZ0f17HRNH39WfL0SYUNOTZ5Lz1TJ8Pajo/87dYGEFlLMm7mIc/k/s6Bvz9HQ== + dependencies: + is-stream "^2.0.0" + type-fest "^0.8.0" + +hast-to-hyperscript@^9.0.0: + version "9.0.1" + resolved "https://registry.npmmirror.com/hast-to-hyperscript/-/hast-to-hyperscript-9.0.1.tgz#9b67fd188e4c81e8ad66f803855334173920218d" + integrity sha512-zQgLKqF+O2F72S1aa4y2ivxzSlko3MAvxkwG8ehGmNiqd98BIN3JM1rAJPmplEyLmGLO2QZYJtIneOSZ2YbJuA== + dependencies: + "@types/unist" "^2.0.3" + comma-separated-tokens "^1.0.0" + property-information "^5.3.0" + space-separated-tokens "^1.0.0" + style-to-object "^0.3.0" + unist-util-is "^4.0.0" + web-namespaces "^1.0.0" + +hast-util-from-parse5@^6.0.0: + version "6.0.1" + resolved "https://registry.npmmirror.com/hast-util-from-parse5/-/hast-util-from-parse5-6.0.1.tgz#554e34abdeea25ac76f5bd950a1f0180e0b3bc2a" + integrity sha512-jeJUWiN5pSxW12Rh01smtVkZgZr33wBokLzKLwinYOUfSzm1Nl/c3GUGebDyOKjdsRgMvoVbV0VpAcpjF4NrJA== + dependencies: + "@types/parse5" "^5.0.0" + hastscript "^6.0.0" + property-information "^5.0.0" + vfile "^4.0.0" + vfile-location "^3.2.0" + web-namespaces "^1.0.0" + +hast-util-is-element@^1.0.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/hast-util-is-element/-/hast-util-is-element-1.1.0.tgz#3b3ed5159a2707c6137b48637fbfe068e175a425" + integrity sha512-oUmNua0bFbdrD/ELDSSEadRVtWZOf3iF6Lbv81naqsIV99RnSCieTbWuWCY8BAeEfKJTKl0gRdokv+dELutHGQ== + +hast-util-parse-selector@^2.0.0: + version "2.2.5" + resolved "https://registry.npmmirror.com/hast-util-parse-selector/-/hast-util-parse-selector-2.2.5.tgz#d57c23f4da16ae3c63b3b6ca4616683313499c3a" + integrity sha512-7j6mrk/qqkSehsM92wQjdIgWM2/BW61u/53G6xmC8i1OmEdKLHbk419QKQUjz6LglWsfqoiHmyMRkP1BGjecNQ== + +hast-util-raw@^6.0.2: + version "6.1.0" + resolved "https://registry.npmmirror.com/hast-util-raw/-/hast-util-raw-6.1.0.tgz#e16a3c2642f65cc7c480c165400a40d604ab75d0" + integrity sha512-5FoZLDHBpka20OlZZ4I/+RBw5piVQ8iI1doEvffQhx5CbCyTtP8UCq8Tw6NmTAMtXgsQxmhW7Ly8OdFre5/YMQ== + dependencies: + "@types/hast" "^2.0.0" + hast-util-from-parse5 "^6.0.0" + hast-util-to-parse5 "^6.0.0" + html-void-elements "^1.0.0" + parse5 "^6.0.0" + unist-util-position "^3.0.0" + unist-util-visit "^2.0.0" + vfile "^4.0.0" + web-namespaces "^1.0.0" + xtend "^4.0.0" + zwitch "^1.0.0" + +hast-util-to-html@^7.1.3: + version "7.1.3" + resolved "https://registry.npmmirror.com/hast-util-to-html/-/hast-util-to-html-7.1.3.tgz#9f339ca9bea71246e565fc79ff7dbfe98bb50f5e" + integrity sha512-yk2+1p3EJTEE9ZEUkgHsUSVhIpCsL/bvT8E5GzmWc+N1Po5gBw+0F8bo7dpxXR0nu0bQVxVZGX2lBGF21CmeDw== + dependencies: + ccount "^1.0.0" + comma-separated-tokens "^1.0.0" + hast-util-is-element "^1.0.0" + hast-util-whitespace "^1.0.0" + html-void-elements "^1.0.0" + property-information "^5.0.0" + space-separated-tokens "^1.0.0" + stringify-entities "^3.0.1" + unist-util-is "^4.0.0" + xtend "^4.0.0" + +hast-util-to-parse5@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/hast-util-to-parse5/-/hast-util-to-parse5-6.0.0.tgz#1ec44650b631d72952066cea9b1445df699f8479" + integrity sha512-Lu5m6Lgm/fWuz8eWnrKezHtVY83JeRGaNQ2kn9aJgqaxvVkFCZQBEhgodZUDUvoodgyROHDb3r5IxAEdl6suJQ== + dependencies: + hast-to-hyperscript "^9.0.0" + property-information "^5.0.0" + web-namespaces "^1.0.0" + xtend "^4.0.0" + zwitch "^1.0.0" + +hast-util-whitespace@^1.0.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/hast-util-whitespace/-/hast-util-whitespace-1.0.4.tgz#e4fe77c4a9ae1cb2e6c25e02df0043d0164f6e41" + integrity sha512-I5GTdSfhYfAPNztx2xJRQpG8cuDSNt599/7YUn7Gx/WxNMsG+a835k97TDkFgk123cwjfwINaZknkKkphx/f2A== + +hastscript@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/hastscript/-/hastscript-6.0.0.tgz#e8768d7eac56c3fdeac8a92830d58e811e5bf640" + integrity sha512-nDM6bvd7lIqDUiYEiu5Sl/+6ReP0BMk/2f4U/Rooccxkj0P5nm+acM5PrGJ/t5I8qPGiqZSE6hVAwZEdZIvP4w== + dependencies: + "@types/hast" "^2.0.0" + comma-separated-tokens "^1.0.0" + hast-util-parse-selector "^2.0.0" + property-information "^5.0.0" + space-separated-tokens "^1.0.0" + +header-case@^2.0.4: + version "2.0.4" + resolved "https://registry.npmmirror.com/header-case/-/header-case-2.0.4.tgz#5a42e63b55177349cf405beb8d775acabb92c063" + integrity sha512-H/vuk5TEEVZwrR0lp2zed9OCo1uAILMlx0JEMgC26rzyJJ3N1v6XkwHHXJQdR2doSjcGPM6OKPYoJgf0plJ11Q== + dependencies: + capital-case "^1.0.4" + tslib "^2.0.3" + +highlight.js@^10.5.0: + version "10.7.3" + resolved "https://registry.npmmirror.com/highlight.js/-/highlight.js-10.7.3.tgz#697272e3991356e40c3cac566a74eef681756531" + integrity sha512-tzcUFauisWKNHaRkN4Wjl/ZA07gENAjFl3J/c480dprkGTg5EQstgaNFqBfUqCq54kZRIEcreTsAgF/m2quD7A== + +hoist-non-react-statics@^3.3.1: + version "3.3.2" + resolved "https://registry.npmmirror.com/hoist-non-react-statics/-/hoist-non-react-statics-3.3.2.tgz#ece0acaf71d62c2969c2ec59feff42a4b1a85b45" + integrity sha512-/gGivxi8JPKWNm/W0jSmzcMPpfpPLc3dY/6GxhX2hQ9iGj3aDfklV4ET7NjKpSinLpJ5vafa9iiGIEZg10SfBw== + dependencies: + react-is "^16.7.0" + +hosted-git-info@^2.1.4: + version "2.8.9" + resolved "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz#dffc0bf9a21c02209090f2aa69429e1414daf3f9" + integrity sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw== + +hosted-git-info@^3.0.8: + version "3.0.8" + resolved "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-3.0.8.tgz#6e35d4cc87af2c5f816e4cb9ce350ba87a3f370d" + integrity sha512-aXpmwoOhRBrw6X3j0h5RloK4x1OzsxMPyxqIHyNfSe2pypkVTZFpEiRoSipPEPlMrh0HW/XsjkJ5WgnCirpNUw== + dependencies: + lru-cache "^6.0.0" + +hosted-git-info@^4.0.1: + version "4.1.0" + resolved "https://registry.npmmirror.com/hosted-git-info/-/hosted-git-info-4.1.0.tgz#827b82867e9ff1c8d0c4d9d53880397d2c86d224" + integrity sha512-kyCuEOWjJqZuDbRHzL8V93NzQhwIB71oFWSyzVo+KPZI+pnQPPxucdkrOZvkLRnrf5URsQM+IJ09Dw29cRALIA== + dependencies: + lru-cache "^6.0.0" + +html-dom-parser@1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/html-dom-parser/-/html-dom-parser-1.0.2.tgz#bb5ff844f214657d899aa4fb7b0a9e7d15607e96" + integrity sha512-Jq4oVkVSn+10ut3fyc2P/Fs1jqTo0l45cP6Q8d2ef/9jfkYwulO0QXmyLI0VUiZrXF4czpGgMEJRa52CQ6Fk8Q== + dependencies: + domhandler "4.2.2" + htmlparser2 "6.1.0" + +html-entities@^1.2.1: + version "1.4.0" + resolved "https://registry.npmmirror.com/html-entities/-/html-entities-1.4.0.tgz#cfbd1b01d2afaf9adca1b10ae7dffab98c71d2dc" + integrity sha512-8nxjcBcd8wovbeKx7h3wTji4e6+rhaVuPNpMqwWgnHh+N9ToqsCs6XztWRBPQ+UtzsoMAdKZtUENoVzU/EMtZA== + +html-entities@^2.3.3: + version "2.3.3" + resolved "https://registry.npmmirror.com/html-entities/-/html-entities-2.3.3.tgz#117d7626bece327fc8baace8868fa6f5ef856e46" + integrity sha512-DV5Ln36z34NNTDgnz0EWGBLZENelNAtkiFA4kyNOG2tDI6Mz1uSWiq1wAKdyjnJwyDiDO7Fa2SO1CTxPXL8VxA== + +html-react-parser@1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/html-react-parser/-/html-react-parser-1.3.0.tgz#81eff0558f34183ac1d372aa9218b8fb47bb3d6d" + integrity sha512-lhpkOFH8pwqEjlNUYCWvjT43/JVCZO9MAZuCS6afT1/VP+bZcNxNUs4AUqiMzH0QPSDHwM/GFNXZNok1KTA4BQ== + dependencies: + domhandler "4.2.2" + html-dom-parser "1.0.2" + react-property "2.0.0" + style-to-js "1.1.0" + +html-tokenize@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/html-tokenize/-/html-tokenize-2.0.1.tgz#c3b2ea6e2837d4f8c06693393e9d2a12c960be5f" + integrity sha512-QY6S+hZ0f5m1WT8WffYN+Hg+xm/w5I8XeUcAq/ZYP5wVC8xbKi4Whhru3FtrAebD5EhBW8rmFzkDI6eCAuFe2w== + dependencies: + buffer-from "~0.1.1" + inherits "~2.0.1" + minimist "~1.2.5" + readable-stream "~1.0.27-1" + through2 "~0.4.1" + +html-void-elements@^1.0.0: + version "1.0.5" + resolved "https://registry.npmmirror.com/html-void-elements/-/html-void-elements-1.0.5.tgz#ce9159494e86d95e45795b166c2021c2cfca4483" + integrity sha512-uE/TxKuyNIcx44cIWnjr/rfIATDH7ZaOMmstu0CwhFG1Dunhlp4OC6/NMbhiwoq5BpW0ubi303qnEk/PZj614w== + +htmlparser2@6.1.0, htmlparser2@^6.1.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/htmlparser2/-/htmlparser2-6.1.0.tgz#c4d762b6c3371a05dbe65e94ae43a9f845fb8fb7" + integrity sha512-gyyPk6rgonLFEDGoeRgQNaEUvdJ4ktTmmUh/h2t7s+M8oPpIPxgNACWa+6ESR57kXstwqPiCut0V8NRpcwgU7A== + dependencies: + domelementtype "^2.0.1" + domhandler "^4.0.0" + domutils "^2.5.2" + entities "^2.0.0" + +htmlparser2@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/htmlparser2/-/htmlparser2-4.1.0.tgz#9a4ef161f2e4625ebf7dfbe6c0a2f52d18a59e78" + integrity sha512-4zDq1a1zhE4gQso/c5LP1OtrhYTncXNSpvJYtWJBtXAETPlMfi3IFNjGuQbYLuVY4ZR0QMqRVvo4Pdy9KLyP8Q== + dependencies: + domelementtype "^2.0.1" + domhandler "^3.0.0" + domutils "^2.0.0" + entities "^2.0.0" + +htmlparser2@^8.0.1: + version "8.0.1" + resolved "https://registry.npmmirror.com/htmlparser2/-/htmlparser2-8.0.1.tgz#abaa985474fcefe269bc761a779b544d7196d010" + integrity sha512-4lVbmc1diZC7GUJQtRQ5yBAeUCL1exyMwmForWkRLnwyzWBFxN633SALPMGYaWZvKe9j1pRZJpauvmxENSp/EA== + dependencies: + domelementtype "^2.3.0" + domhandler "^5.0.2" + domutils "^3.0.1" + entities "^4.3.0" + +http-cache-semantics@^4.0.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/http-cache-semantics/-/http-cache-semantics-4.1.0.tgz#49e91c5cbf36c9b94bcfcd71c23d5249ec74e390" + integrity sha512-carPklcUh7ROWRK7Cv27RPtdhYhUsela/ue5/jKzjegVvXDqM2ILE9Q2BGn9JZJh1g87cp56su/FgQSzcWS8cQ== + +http-errors@1.8.0: + version "1.8.0" + resolved "https://registry.npmmirror.com/http-errors/-/http-errors-1.8.0.tgz#75d1bbe497e1044f51e4ee9e704a62f28d336507" + integrity sha512-4I8r0C5JDhT5VkvI47QktDW75rNlGVsUf/8hzjCC/wkWI/jdTRmBb9aI7erSG82r1bjKY3F6k28WnsVxB1C73A== + dependencies: + depd "~1.1.2" + inherits "2.0.4" + setprototypeof "1.2.0" + statuses ">= 1.5.0 < 2" + toidentifier "1.0.0" + +http-errors@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/http-errors/-/http-errors-2.0.0.tgz#b7774a1486ef73cf7667ac9ae0858c012c57b9d3" + integrity sha512-FtwrG/euBzaEjYeRqOgly7G0qviiXoJWnvEH2Z1plBdXgbyjv34pHTSb9zoeHMyDy33+DWy5Wt9Wo+TURtOYSQ== + dependencies: + depd "2.0.0" + inherits "2.0.4" + setprototypeof "1.2.0" + statuses "2.0.1" + toidentifier "1.0.1" + +http-proxy@^1.18.1: + version "1.18.1" + resolved "https://registry.npmmirror.com/http-proxy/-/http-proxy-1.18.1.tgz#401541f0534884bbf95260334e72f88ee3976549" + integrity sha512-7mz/721AbnJwIVbnaSv1Cz3Am0ZLT/UBwkC92VlxhXv/k/BBQfM2fXElQNC27BVGr0uwUpplYPQM9LnaBMR5NQ== + dependencies: + eventemitter3 "^4.0.0" + follow-redirects "^1.0.0" + requires-port "^1.0.0" + +http-signature@~1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/http-signature/-/http-signature-1.2.0.tgz#9aecd925114772f3d95b65a60abb8f7c18fbace1" + integrity sha512-CAbnr6Rz4CYQkLYUtSNXxQPUH2gK8f3iWexVlsnMeD+GjlsQ0Xsy1cOX+mN3dtxYomRy21CiOzU8Uhw6OwncEQ== + dependencies: + assert-plus "^1.0.0" + jsprim "^1.2.2" + sshpk "^1.7.0" + +http2-wrapper@^1.0.0-beta.5.2: + version "1.0.3" + resolved "https://registry.npmmirror.com/http2-wrapper/-/http2-wrapper-1.0.3.tgz#b8f55e0c1f25d4ebd08b3b0c2c079f9590800b3d" + integrity sha512-V+23sDMr12Wnz7iTcDeJr3O6AIxlnvT/bmaAAAP/Xda35C90p9599p0F1eHR/N1KILWSoWVAiOMFjBBXaXSMxg== + dependencies: + quick-lru "^5.1.1" + resolve-alpn "^1.0.0" + +human-signals@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/human-signals/-/human-signals-2.1.0.tgz#dc91fcba42e4d06e4abaed33b3e7a3c02f514ea0" + integrity sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw== + +iconv-lite@0.4.24, iconv-lite@^0.4.24, iconv-lite@^0.4.4: + version "0.4.24" + resolved "https://registry.npmmirror.com/iconv-lite/-/iconv-lite-0.4.24.tgz#2022b4b25fbddc21d2f524974a474aafe733908b" + integrity sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA== + dependencies: + safer-buffer ">= 2.1.2 < 3" + +icss-utils@^5.0.0, icss-utils@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/icss-utils/-/icss-utils-5.1.0.tgz#c6be6858abd013d768e98366ae47e25d5887b1ae" + integrity sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA== + +idb-keyval@^3.2.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/idb-keyval/-/idb-keyval-3.2.0.tgz#cbbf354deb5684b6cdc84376294fc05932845bd6" + integrity sha512-slx8Q6oywCCSfKgPgL0sEsXtPVnSbTLWpyiDcu6msHOyKOLari1TD1qocXVCft80umnkk3/Qqh3lwoFt8T/BPQ== + +ieee754@^1.1.13, ieee754@^1.2.1: + version "1.2.1" + resolved "https://registry.npmmirror.com/ieee754/-/ieee754-1.2.1.tgz#8eb7a10a63fff25d15a57b001586d177d1b0d352" + integrity sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA== + +ignore@^4.0.6: + version "4.0.6" + resolved "https://registry.npmmirror.com/ignore/-/ignore-4.0.6.tgz#750e3db5862087b4737ebac8207ffd1ef27b25fc" + integrity sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg== + +ignore@^5.1.4, ignore@^5.1.8, ignore@^5.2.0: + version "5.2.0" + resolved "https://registry.npmmirror.com/ignore/-/ignore-5.2.0.tgz#6d3bac8fa7fe0d45d9f9be7bac2fc279577e345a" + integrity sha512-CmxgYGiEPCLhfLnpPp1MoRmifwEIOgjcHXxOBjv7mY96c+eWScsOP9c112ZyLdWHi0FxHjI+4uVhKYp/gcdRmQ== + +image-q@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/image-q/-/image-q-4.0.0.tgz#31e075be7bae3c1f42a85c469b4732c358981776" + integrity sha512-PfJGVgIfKQJuq3s0tTDOKtztksibuUEbJQIYT3by6wctQo+Rdlh7ef4evJ5NCdxY4CfMbvFkocEwbl4BF8RlJw== + dependencies: + "@types/node" "16.9.1" + +immer@^9.0.7: + version "9.0.15" + resolved "https://registry.npmmirror.com/immer/-/immer-9.0.15.tgz#0b9169e5b1d22137aba7d43f8a81a495dd1b62dc" + integrity sha512-2eB/sswms9AEUSkOm4SbV5Y7Vmt/bKRwByd52jfLkW4OLYeaTP3EEiJ9agqU0O/tq6Dk62Zfj+TJSqfm1rLVGQ== + +immutable@~3.7.6: + version "3.7.6" + resolved "https://registry.npmmirror.com/immutable/-/immutable-3.7.6.tgz#13b4d3cb12befa15482a26fe1b2ebae640071e4b" + integrity sha512-AizQPcaofEtO11RZhPPHBOJRdo/20MKQF9mBLnVkBoyHi1/zXK8fzVdnEpSV9gxqtnh6Qomfp3F0xT5qP/vThw== + +import-fresh@^3.0.0, import-fresh@^3.1.0, import-fresh@^3.2.1: + version "3.3.0" + resolved "https://registry.npmmirror.com/import-fresh/-/import-fresh-3.3.0.tgz#37162c25fcb9ebaa2e6e53d5b4d88ce17d9e0c2b" + integrity sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw== + dependencies: + parent-module "^1.0.0" + resolve-from "^4.0.0" + +import-from@3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/import-from/-/import-from-3.0.0.tgz#055cfec38cd5a27d8057ca51376d7d3bf0891966" + integrity sha512-CiuXOFFSzkU5x/CR0+z7T91Iht4CXgfCxVOFRhh2Zyhg5wOpWvvDLQUsWl+gcN+QscYBjez8hDCt85O7RLDttQ== + dependencies: + resolve-from "^5.0.0" + +import-from@4.0.0, import-from@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/import-from/-/import-from-4.0.0.tgz#2710b8d66817d232e16f4166e319248d3d5492e2" + integrity sha512-P9J71vT5nLlDeV8FHs5nNxaLbrpfAV5cF5srvbZfpwpcJoM/xZR3hiv+q+SAnuSmuGbXMWud063iIMx/V/EWZQ== + +import-lazy@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/import-lazy/-/import-lazy-2.1.0.tgz#05698e3d45c88e8d7e9d92cb0584e77f096f3e43" + integrity sha512-m7ZEHgtw69qOGw+jwxXkHlrlIPdTGkyh66zXZ1ajZbxkDBNjSY/LGbmjc7h0s2ELsUDTAhFr55TrPSSqJGPG0A== + +imurmurhash@^0.1.4: + version "0.1.4" + resolved "https://registry.npmmirror.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" + integrity sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA== + +indent-string@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/indent-string/-/indent-string-4.0.0.tgz#624f8f4497d619b2d9768531d58f4122854d7251" + integrity sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg== + +inflight@^1.0.4: + version "1.0.6" + resolved "https://registry.npmmirror.com/inflight/-/inflight-1.0.6.tgz#49bd6331d7d02d0c09bc910a1075ba8165b56df9" + integrity sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA== + dependencies: + once "^1.3.0" + wrappy "1" + +inherits@2, inherits@2.0.4, inherits@^2.0.0, inherits@^2.0.3, inherits@^2.0.4, inherits@~2.0.1, inherits@~2.0.3: + version "2.0.4" + resolved "https://registry.npmmirror.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" + integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== + +ini@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/ini/-/ini-2.0.0.tgz#e5fd556ecdd5726be978fa1001862eacb0a94bc5" + integrity sha512-7PnF4oN3CvZF23ADhA5wRaYEQpJ8qygSkbtTXWBeXWXmEVRXK+1ITciHWwHhsjv1TmW0MgacIv6hEi5pX5NQdA== + +ini@^1.3.5, ini@~1.3.0: + version "1.3.8" + resolved "https://registry.npmmirror.com/ini/-/ini-1.3.8.tgz#a29da425b48806f34767a4efce397269af28432c" + integrity sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew== + +inline-style-parser@0.1.1: + version "0.1.1" + resolved "https://registry.npmmirror.com/inline-style-parser/-/inline-style-parser-0.1.1.tgz#ec8a3b429274e9c0a1f1c4ffa9453a7fef72cea1" + integrity sha512-7NXolsK4CAS5+xvdj5OMMbI962hU/wvwoxk+LWR9Ek9bVtyuuYScDN6eS0rUm6TxApFpw7CX1o4uJzcd4AyD3Q== + +inquirer@^7.0.0: + version "7.3.3" + resolved "https://registry.npmmirror.com/inquirer/-/inquirer-7.3.3.tgz#04d176b2af04afc157a83fd7c100e98ee0aad003" + integrity sha512-JG3eIAj5V9CwcGvuOmoo6LB9kbAYT8HXffUl6memuszlwDC/qvFAJw49XJ5NROSFNPxp3iQg1GqkFhaY/CR0IA== + dependencies: + ansi-escapes "^4.2.1" + chalk "^4.1.0" + cli-cursor "^3.1.0" + cli-width "^3.0.0" + external-editor "^3.0.3" + figures "^3.0.0" + lodash "^4.17.19" + mute-stream "0.0.8" + run-async "^2.4.0" + rxjs "^6.6.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + through "^2.3.6" + +internal-slot@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/internal-slot/-/internal-slot-1.0.3.tgz#7347e307deeea2faac2ac6205d4bc7d34967f59c" + integrity sha512-O0DB1JC/sPyZl7cIo78n5dR7eUSwwpYPiXRhTzNxZVAMUuB8vlnRFyLxdrVToks6XPLVnFfbzaVd5WLjhgg+vA== + dependencies: + get-intrinsic "^1.1.0" + has "^1.0.3" + side-channel "^1.0.4" + +invariant@^2.2.3, invariant@^2.2.4: + version "2.2.4" + resolved "https://registry.npmmirror.com/invariant/-/invariant-2.2.4.tgz#610f3c92c9359ce1db616e538008d23ff35158e6" + integrity sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA== + dependencies: + loose-envify "^1.0.0" + +ipaddr.js@1.9.1: + version "1.9.1" + resolved "https://registry.npmmirror.com/ipaddr.js/-/ipaddr.js-1.9.1.tgz#bff38543eeb8984825079ff3a2a8e6cbd46781b3" + integrity sha512-0KI/607xoxSToH7GjN1FfSbLoU0+btTicjsQSWQlh/hZykN8KpmMf7uYwPW3R+akZ6R/w18ZlXSHBYXiYUPO3g== + +is-absolute-url@^3.0.0: + version "3.0.3" + resolved "https://registry.npmmirror.com/is-absolute-url/-/is-absolute-url-3.0.3.tgz#96c6a22b6a23929b11ea0afb1836c36ad4a5d698" + integrity sha512-opmNIX7uFnS96NtPmhWQgQx6/NYFgsUXYMllcfzwWKUMwfo8kku1TvE6hkNcH+Q1ts5cMVrsY7j0bxXQDciu9Q== + +is-absolute@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-absolute/-/is-absolute-1.0.0.tgz#395e1ae84b11f26ad1795e73c17378e48a301576" + integrity sha512-dOWoqflvcydARa360Gvv18DZ/gRuHKi2NU/wU5X1ZFzdYfH29nkiNZsF3mp4OJ3H4yo9Mx8A/uAGNzpzPN3yBA== + dependencies: + is-relative "^1.0.0" + is-windows "^1.0.1" + +is-alphabetical@^1.0.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-alphabetical/-/is-alphabetical-1.0.4.tgz#9e7d6b94916be22153745d184c298cbf986a686d" + integrity sha512-DwzsA04LQ10FHTZuL0/grVDk4rFoVH1pjAToYwBrHSxcrBIGQuXrQMtD5U1b0U2XVgKZCTLLP8u2Qxqhy3l2Vg== + +is-alphanumerical@^1.0.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-alphanumerical/-/is-alphanumerical-1.0.4.tgz#7eb9a2431f855f6b1ef1a78e326df515696c4dbf" + integrity sha512-UzoZUr+XfVz3t3v4KyGEniVL9BDRoQtY7tOyrRybkVNjDFWyo1yhXNGrrBTQxp3ib9BLAWs7k2YKBQsFRkZG9A== + dependencies: + is-alphabetical "^1.0.0" + is-decimal "^1.0.0" + +is-arrayish@^0.2.1: + version "0.2.1" + resolved "https://registry.npmmirror.com/is-arrayish/-/is-arrayish-0.2.1.tgz#77c99840527aa8ecb1a8ba697b80645a7a926a9d" + integrity sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg== + +is-arrayish@^0.3.1: + version "0.3.2" + resolved "https://registry.npmmirror.com/is-arrayish/-/is-arrayish-0.3.2.tgz#4574a2ae56f7ab206896fb431eaeed066fdf8f03" + integrity sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ== + +is-bigint@^1.0.1: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-bigint/-/is-bigint-1.0.4.tgz#08147a1875bc2b32005d41ccd8291dffc6691df3" + integrity sha512-zB9CruMamjym81i2JZ3UMn54PKGsQzsJeo6xvN3HJJ4CAsQNB6iRutp2To77OfCNuoxspsIhzaPoO1zyCEhFOg== + dependencies: + has-bigints "^1.0.1" + +is-binary-path@~2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/is-binary-path/-/is-binary-path-2.1.0.tgz#ea1f7f3b80f064236e83470f86c09c254fb45b09" + integrity sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw== + dependencies: + binary-extensions "^2.0.0" + +is-boolean-object@^1.1.0: + version "1.1.2" + resolved "https://registry.npmmirror.com/is-boolean-object/-/is-boolean-object-1.1.2.tgz#5c6dc200246dd9321ae4b885a114bb1f75f63719" + integrity sha512-gDYaKHJmnj4aWxyj6YHyXVpdQawtVLHU5cb+eztPGczf6cjuTdwve5ZIEfgXqH4e57An1D1AKf8CZ3kYrQRqYA== + dependencies: + call-bind "^1.0.2" + has-tostringtag "^1.0.0" + +is-buffer@^2.0.0: + version "2.0.5" + resolved "https://registry.npmmirror.com/is-buffer/-/is-buffer-2.0.5.tgz#ebc252e400d22ff8d77fa09888821a24a658c191" + integrity sha512-i2R6zNFDwgEHJyQUtJEk0XFi1i0dPFn/oqjK3/vPCcDeJvW5NQ83V8QbicfF1SupOaB0h8ntgBC2YiE7dfyctQ== + +is-callable@^1.1.4, is-callable@^1.2.4: + version "1.2.4" + resolved "https://registry.npmmirror.com/is-callable/-/is-callable-1.2.4.tgz#47301d58dd0259407865547853df6d61fe471945" + integrity sha512-nsuwtxZfMX67Oryl9LCQ+upnC0Z0BgpwntpS89m1H/TLF0zNfzfLMV/9Wa/6MZsj0acpEjAO0KF1xT6ZdLl95w== + +is-ci@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/is-ci/-/is-ci-2.0.0.tgz#6bc6334181810e04b5c22b3d589fdca55026404c" + integrity sha512-YfJT7rkpQB0updsdHLGWrvhBJfcfzNNawYDNIyQXJz0IViGf75O8EBPKSdvw2rF+LGCsX4FZ8tcr3b19LcZq4w== + dependencies: + ci-info "^2.0.0" + +is-core-module@^2.2.0, is-core-module@^2.5.0, is-core-module@^2.8.1: + version "2.9.0" + resolved "https://registry.npmmirror.com/is-core-module/-/is-core-module-2.9.0.tgz#e1c34429cd51c6dd9e09e0799e396e27b19a9c69" + integrity sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A== + dependencies: + has "^1.0.3" + +is-date-object@^1.0.1: + version "1.0.5" + resolved "https://registry.npmmirror.com/is-date-object/-/is-date-object-1.0.5.tgz#0841d5536e724c25597bf6ea62e1bd38298df31f" + integrity sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ== + dependencies: + has-tostringtag "^1.0.0" + +is-decimal@^1.0.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-decimal/-/is-decimal-1.0.4.tgz#65a3a5958a1c5b63a706e1b333d7cd9f630d3fa5" + integrity sha512-RGdriMmQQvZ2aqaQq3awNA6dCGtKpiDFcOzrTWrDAT2MiWrKQVPmxLGHl7Y2nNu6led0kEyoX0enY0qXYsv9zw== + +is-docker@^2.0.0, is-docker@^2.1.1, is-docker@^2.2.1: + version "2.2.1" + resolved "https://registry.npmmirror.com/is-docker/-/is-docker-2.2.1.tgz#33eeabe23cfe86f14bde4408a02c0cfb853acdaa" + integrity sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ== + +is-extendable@^0.1.0: + version "0.1.1" + resolved "https://registry.npmmirror.com/is-extendable/-/is-extendable-0.1.1.tgz#62b110e289a471418e3ec36a617d472e301dfc89" + integrity sha512-5BMULNob1vgFX6EjQw5izWDxrecWK9AM72rugNr0TFldMOi0fj6Jk+zeKIt0xGj4cEfQIJth4w3OKWOJ4f+AFw== + +is-extglob@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-extglob/-/is-extglob-1.0.0.tgz#ac468177c4943405a092fc8f29760c6ffc6206c0" + integrity sha512-7Q+VbVafe6x2T+Tu6NcOf6sRklazEPmBoB3IWk3WdGZM2iGUwU/Oe3Wtq5lSEkDTTlpp8yx+5t4pzO/i9Ty1ww== + +is-extglob@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2" + integrity sha512-SbKbANkN603Vi4jEZv49LeVJMn4yGwsbzZworEoyEiutsN3nJYdbO36zfhGJ6QEDpOZIFkDtnq5JRxmvl3jsoQ== + +is-fullwidth-code-point@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz#ef9e31386f031a7f0d643af82fde50c457ef00cb" + integrity sha512-1pqUqRjkhPJ9miNq9SwMfdvi6lBJcd6eFxvfaivQhaH3SgisfiuudvFntdKOmxuee/77l+FPjKrQjWvmPjWrRw== + dependencies: + number-is-nan "^1.0.0" + +is-fullwidth-code-point@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz#a3b30a5c4f199183167aaab93beefae3ddfb654f" + integrity sha512-VHskAKYM8RfSFXwee5t5cbN5PZeq1Wrh6qd5bkyiXIf6UQcN6w/A0eXM9r6t8d+GYOh+o6ZhiEnb88LN/Y8m2w== + +is-fullwidth-code-point@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" + integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== + +is-function@^1.0.1: + version "1.0.2" + resolved "https://registry.npmmirror.com/is-function/-/is-function-1.0.2.tgz#4f097f30abf6efadac9833b17ca5dc03f8144e08" + integrity sha512-lw7DUp0aWXYg+CBCN+JKkcE0Q2RayZnSvnZBlwgxHBQhqt5pZNVy4Ri7H9GmmXkdu7LUthszM+Tor1u/2iBcpQ== + +is-glob@4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/is-glob/-/is-glob-4.0.1.tgz#7567dbe9f2f5e2467bc77ab83c4a29482407a5dc" + integrity sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg== + dependencies: + is-extglob "^2.1.1" + +is-glob@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/is-glob/-/is-glob-2.0.1.tgz#d096f926a3ded5600f3fdfd91198cb0888c2d863" + integrity sha512-a1dBeB19NXsf/E0+FHqkagizel/LQw2DjSQpvQrj3zT+jYPpaUCryPnrQajXKFLCMuf4I6FhRpaGtw4lPrG6Eg== + dependencies: + is-extglob "^1.0.0" + +is-glob@^4.0.0, is-glob@^4.0.1, is-glob@^4.0.3, is-glob@~4.0.1: + version "4.0.3" + resolved "https://registry.npmmirror.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" + integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg== + dependencies: + is-extglob "^2.1.1" + +is-hexadecimal@^1.0.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-hexadecimal/-/is-hexadecimal-1.0.4.tgz#cc35c97588da4bd49a8eedd6bc4082d44dcb23a7" + integrity sha512-gyPJuv83bHMpocVYoqof5VDiZveEoGoFL8m3BXNb2VW8Xs+rz9kqO8LOQ5DH6EsuvilT1ApazU0pyl+ytbPtlw== + +is-installed-globally@^0.4.0: + version "0.4.0" + resolved "https://registry.npmmirror.com/is-installed-globally/-/is-installed-globally-0.4.0.tgz#9a0fd407949c30f86eb6959ef1b7994ed0b7b520" + integrity sha512-iwGqO3J21aaSkC7jWnHP/difazwS7SFeIqxv6wEtLU8Y5KlzFTjyqcSIT0d8s4+dDhKytsk9PJZ2BkS5eZwQRQ== + dependencies: + global-dirs "^3.0.0" + is-path-inside "^3.0.2" + +is-invalid-path@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/is-invalid-path/-/is-invalid-path-0.1.0.tgz#307a855b3cf1a938b44ea70d2c61106053714f34" + integrity sha512-aZMG0T3F34mTg4eTdszcGXx54oiZ4NtHSft3hWNJMGJXUUqdIj3cOZuHcU0nCWWcY3jd7yRe/3AEm3vSNTpBGQ== + dependencies: + is-glob "^2.0.0" + +is-lower-case@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/is-lower-case/-/is-lower-case-2.0.2.tgz#1c0884d3012c841556243483aa5d522f47396d2a" + integrity sha512-bVcMJy4X5Og6VZfdOZstSexlEy20Sr0k/p/b2IlQJlfdKAQuMpiv5w2Ccxb8sKdRUNAG1PnHVHjFSdRDVS6NlQ== + dependencies: + tslib "^2.0.3" + +is-negative-zero@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/is-negative-zero/-/is-negative-zero-2.0.2.tgz#7bf6f03a28003b8b3965de3ac26f664d765f3150" + integrity sha512-dqJvarLawXsFbNDeJW7zAz8ItJ9cd28YufuuFzh0G8pNHjJMnY08Dv7sYX2uF5UpQOwieAeOExEYAWWfu7ZZUA== + +is-npm@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/is-npm/-/is-npm-5.0.0.tgz#43e8d65cc56e1b67f8d47262cf667099193f45a8" + integrity sha512-WW/rQLOazUq+ST/bCAVBp/2oMERWLsR7OrKyt052dNDk4DHcDE0/7QSXITlmi+VBcV13DfIbysG3tZJm5RfdBA== + +is-number-object@^1.0.4: + version "1.0.7" + resolved "https://registry.npmmirror.com/is-number-object/-/is-number-object-1.0.7.tgz#59d50ada4c45251784e9904f5246c742f07a42fc" + integrity sha512-k1U0IRzLMo7ZlYIfzRu23Oh6MiIFasgpb9X76eqfFZAqwH44UI4KTBvBYIZ1dSL9ZzChTB9ShHfLkR4pdW5krQ== + dependencies: + has-tostringtag "^1.0.0" + +is-number@^7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" + integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== + +is-obj@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/is-obj/-/is-obj-1.0.1.tgz#3e4729ac1f5fde025cd7d83a896dab9f4f67db0f" + integrity sha512-l4RyHgRqGN4Y3+9JHVrNqO+tN0rV5My76uW5/nuO4K1b6vw5G8d/cmFjP9tRfEsdhZNt0IFdZuK/c2Vr4Nb+Qg== + +is-obj@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/is-obj/-/is-obj-2.0.0.tgz#473fb05d973705e3fd9620545018ca8e22ef4982" + integrity sha512-drqDG3cbczxxEJRoOXcOjtdp1J/lyp1mNn0xaznRs8+muBhgQcrnbspox5X5fOw0HnMnbfDzvnEMEtqDEJEo8w== + +is-path-inside@^3.0.2: + version "3.0.3" + resolved "https://registry.npmmirror.com/is-path-inside/-/is-path-inside-3.0.3.tgz#d231362e53a07ff2b0e0ea7fed049161ffd16283" + integrity sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ== + +is-plain-obj@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-1.1.0.tgz#71a50c8429dfca773c92a390a4a03b39fcd51d3e" + integrity sha512-yvkRyxmFKEOQ4pNXCmJG5AEQNlXJS5LaONXo5/cLdTZdWvsZ1ioJEonLGAosKlMWE8lwUy/bJzMjcw8az73+Fg== + +is-plain-obj@^2.0.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/is-plain-obj/-/is-plain-obj-2.1.0.tgz#45e42e37fccf1f40da8e5f76ee21515840c09287" + integrity sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA== + +is-plain-object@^2.0.4: + version "2.0.4" + resolved "https://registry.npmmirror.com/is-plain-object/-/is-plain-object-2.0.4.tgz#2c163b3fafb1b606d9d17928f05c2a1c38e07677" + integrity sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og== + dependencies: + isobject "^3.0.1" + +is-promise@4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/is-promise/-/is-promise-4.0.0.tgz#42ff9f84206c1991d26debf520dd5c01042dd2f3" + integrity sha512-hvpoI6korhJMnej285dSg6nu1+e6uxs7zG3BYAm5byqDsgJNWwxzM6z6iZiAgQR4TJ30JmBTOwqZUw3WlyH3AQ== + +is-promise@^2.2.2: + version "2.2.2" + resolved "https://registry.npmmirror.com/is-promise/-/is-promise-2.2.2.tgz#39ab959ccbf9a774cf079f7b40c7a26f763135f1" + integrity sha512-+lP4/6lKUBfQjZ2pdxThZvLUAafmZb8OAxFb8XXtiQmS35INgr85hdOGoEs124ez1FCnZJt6jau/T+alh58QFQ== + +is-regex@^1.1.4: + version "1.1.4" + resolved "https://registry.npmmirror.com/is-regex/-/is-regex-1.1.4.tgz#eef5663cd59fa4c0ae339505323df6854bb15958" + integrity sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg== + dependencies: + call-bind "^1.0.2" + has-tostringtag "^1.0.0" + +is-regexp@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-regexp/-/is-regexp-1.0.0.tgz#fd2d883545c46bac5a633e7b9a09e87fa2cb5069" + integrity sha512-7zjFAPO4/gwyQAAgRRmqeEeyIICSdmCqa3tsVHMdBzaXXRiqopZL4Cyghg/XulGWrtABTpbnYYzzIRffLkP4oA== + +is-relative-url@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/is-relative-url/-/is-relative-url-3.0.0.tgz#f623c8e26baa5bd3742b3b7ec074f50f3b45b3f3" + integrity sha512-U1iSYRlY2GIMGuZx7gezlB5dp1Kheaym7zKzO1PV06mOihiWTXejLwm4poEJysPyXF+HtK/BEd0DVlcCh30pEA== + dependencies: + is-absolute-url "^3.0.0" + +is-relative@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-relative/-/is-relative-1.0.0.tgz#a1bb6935ce8c5dba1e8b9754b9b2dcc020e2260d" + integrity sha512-Kw/ReK0iqwKeu0MITLFuj0jbPAmEiOsIwyIXvvbfa6QfmN9pkD1M+8pdk7Rl/dTKbH34/XBFMbgD4iMJhLQbGA== + dependencies: + is-unc-path "^1.0.0" + +is-root@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/is-root/-/is-root-2.1.0.tgz#809e18129cf1129644302a4f8544035d51984a9c" + integrity sha512-AGOriNp96vNBd3HtU+RzFEc75FfR5ymiYv8E553I71SCeXBiMsVDUtdio1OEFvrPyLIQ9tVR5RxXIFe5PUFjMg== + +is-shared-array-buffer@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/is-shared-array-buffer/-/is-shared-array-buffer-1.0.2.tgz#8f259c573b60b6a32d4058a1a07430c0a7344c79" + integrity sha512-sqN2UDu1/0y6uvXyStCOzyhAjCSlHceFoMKJW8W9EU9cvic/QdsZ0kEU93HEy3IUEFZIiH/3w+AH/UQbPHNdhA== + dependencies: + call-bind "^1.0.2" + +is-ssh@^1.3.0: + version "1.3.3" + resolved "https://registry.npmmirror.com/is-ssh/-/is-ssh-1.3.3.tgz#7f133285ccd7f2c2c7fc897b771b53d95a2b2c7e" + integrity sha512-NKzJmQzJfEEma3w5cJNcUMxoXfDjz0Zj0eyCalHn2E6VOwlzjZo0yuO2fcBSf8zhFuVCL/82/r5gRcoi6aEPVQ== + dependencies: + protocols "^1.1.0" + +is-stream@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/is-stream/-/is-stream-1.1.0.tgz#12d4a3dd4e68e0b79ceb8dbc84173ae80d91ca44" + integrity sha512-uQPm8kcs47jx38atAcWTVxyltQYoPT68y9aWYdV6yWXSyW8mzSat0TL6CiWdZeCdF3KrAvpVtnHbTv4RN+rqdQ== + +is-stream@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077" + integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg== + +is-string@^1.0.5, is-string@^1.0.7: + version "1.0.7" + resolved "https://registry.npmmirror.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" + integrity sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg== + dependencies: + has-tostringtag "^1.0.0" + +is-symbol@^1.0.2, is-symbol@^1.0.3: + version "1.0.4" + resolved "https://registry.npmmirror.com/is-symbol/-/is-symbol-1.0.4.tgz#a6dac93b635b063ca6872236de88910a57af139c" + integrity sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg== + dependencies: + has-symbols "^1.0.2" + +is-typedarray@^1.0.0, is-typedarray@~1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-typedarray/-/is-typedarray-1.0.0.tgz#e479c80858df0c1b11ddda6940f96011fcda4a9a" + integrity sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA== + +is-unc-path@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/is-unc-path/-/is-unc-path-1.0.0.tgz#d731e8898ed090a12c352ad2eaed5095ad322c9d" + integrity sha512-mrGpVd0fs7WWLfVsStvgF6iEJnbjDFZh9/emhRDcGWTduTfNHd9CHeUwH3gYIjdbwo4On6hunkztwOaAw0yllQ== + dependencies: + unc-path-regex "^0.1.2" + +is-upper-case@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/is-upper-case/-/is-upper-case-2.0.2.tgz#f1105ced1fe4de906a5f39553e7d3803fd804649" + integrity sha512-44pxmxAvnnAOwBg4tHPnkfvgjPwbc5QIsSstNU+YcJ1ovxVzCWpSGosPJOZh/a1tdl81fbgnLc9LLv+x2ywbPQ== + dependencies: + tslib "^2.0.3" + +is-valid-domain@^0.1.6: + version "0.1.6" + resolved "https://registry.npmmirror.com/is-valid-domain/-/is-valid-domain-0.1.6.tgz#3c85469d2938f170c8f82ce6e52df8ad9fca8105" + integrity sha512-ZKtq737eFkZr71At8NxOFcP9O1K89gW3DkdrGMpp1upr/ueWjj+Weh4l9AI4rN0Gt8W2M1w7jrG2b/Yv83Ljpg== + dependencies: + punycode "^2.1.1" + +is-valid-path@^0.1.1: + version "0.1.1" + resolved "https://registry.npmmirror.com/is-valid-path/-/is-valid-path-0.1.1.tgz#110f9ff74c37f663e1ec7915eb451f2db93ac9df" + integrity sha512-+kwPrVDu9Ms03L90Qaml+79+6DZHqHyRoANI6IsZJ/g8frhnfchDOBCa0RbQ6/kdHt5CS5OeIEyrYznNuVN+8A== + dependencies: + is-invalid-path "^0.1.0" + +is-weakref@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/is-weakref/-/is-weakref-1.0.2.tgz#9529f383a9338205e89765e0392efc2f100f06f2" + integrity sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ== + dependencies: + call-bind "^1.0.2" + +is-windows@^1.0.1: + version "1.0.2" + resolved "https://registry.npmmirror.com/is-windows/-/is-windows-1.0.2.tgz#d1850eb9791ecd18e6182ce12a30f396634bb19d" + integrity sha512-eXK1UInq2bPmjyX6e3VHIzMLobc4J94i4AWn+Hpq3OU5KkrRC96OAcR3PRJ/pGu6m8TRnBHP9dkXQVsT/COVIA== + +is-wsl@^2.1.1, is-wsl@^2.2.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/is-wsl/-/is-wsl-2.2.0.tgz#74a4c76e77ca9fd3f932f290c17ea326cd157271" + integrity sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww== + dependencies: + is-docker "^2.0.0" + +is-yarn-global@^0.3.0: + version "0.3.0" + resolved "https://registry.npmmirror.com/is-yarn-global/-/is-yarn-global-0.3.0.tgz#d502d3382590ea3004893746754c89139973e232" + integrity sha512-VjSeb/lHmkoyd8ryPVIKvOCn4D1koMqY+vqyjjUfc3xyKtP4dYOxM44sZrnqQSzSds3xyOrUTLTC9LVCVgLngw== + +isarray@0.0.1: + version "0.0.1" + resolved "https://registry.npmmirror.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" + integrity sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ== + +isarray@~1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" + integrity sha512-VLghIWNM6ELQzo7zwmcg0NmTVyWKYjvIeM83yjp0wRDTmUnrM678fQbcKBo6n2CJEF0szoG//ytg+TKla89ALQ== + +isexe@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" + integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== + +isobject@^3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/isobject/-/isobject-3.0.1.tgz#4e431e92b11a9731636aa1f9c8d1ccbcfdab78df" + integrity sha512-WhB9zCku7EGTj/HQQRz5aUQEUeoQZH2bWcltRErOpymJ4boYE6wL9Tbr23krRPSZ+C5zqNSrSw+Cc7sZZ4b7vg== + +isomorphic-ws@4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/isomorphic-ws/-/isomorphic-ws-4.0.1.tgz#55fd4cd6c5e6491e76dc125938dd863f5cd4f2dc" + integrity sha512-BhBvN2MBpWTaSHdWRb/bwdZJ1WaehQ2L1KngkCkfLUGF0mAWAT1sQUQacEmQ0jXkFw/czDXPNQSL5u2/Krsz1w== + +isstream@~0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/isstream/-/isstream-0.1.2.tgz#47e63f7af55afa6f92e1500e690eb8b8529c099a" + integrity sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g== + +iterall@^1.2.1: + version "1.3.0" + resolved "https://registry.npmmirror.com/iterall/-/iterall-1.3.0.tgz#afcb08492e2915cbd8a0884eb93a8c94d0d72fea" + integrity sha512-QZ9qOMdF+QLHxy1QIpUHUU1D5pS2CG2P69LF6L6CPjPYA/XMOmKV3PZpawHoAjHNyB0swdVTRxdYT4tbBbxqwg== + +jest-worker@^26.3.0: + version "26.6.2" + resolved "https://registry.npmmirror.com/jest-worker/-/jest-worker-26.6.2.tgz#7f72cbc4d643c365e27b9fd775f9d0eaa9c7a8ed" + integrity sha512-KWYVV1c4i+jbMpaBC+U++4Va0cp8OisU185o73T1vo99hqi7w8tSJfUXYswwqqrjzwxa6KpRK54WhPvwf5w6PQ== + dependencies: + "@types/node" "*" + merge-stream "^2.0.0" + supports-color "^7.0.0" + +jest-worker@^27.3.1, jest-worker@^27.4.5: + version "27.5.1" + resolved "https://registry.npmmirror.com/jest-worker/-/jest-worker-27.5.1.tgz#8d146f0900e8973b106b6f73cc1e9a8cb86f8db0" + integrity sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg== + dependencies: + "@types/node" "*" + merge-stream "^2.0.0" + supports-color "^8.0.0" + +jimp@^0.16.1: + version "0.16.1" + resolved "https://registry.npmmirror.com/jimp/-/jimp-0.16.1.tgz#192f851a30e5ca11112a3d0aa53137659a78ca7a" + integrity sha512-+EKVxbR36Td7Hfd23wKGIeEyHbxShZDX6L8uJkgVW3ESA9GiTEPK08tG1XI2r/0w5Ch0HyJF5kPqF9K7EmGjaw== + dependencies: + "@babel/runtime" "^7.7.2" + "@jimp/custom" "^0.16.1" + "@jimp/plugins" "^0.16.1" + "@jimp/types" "^0.16.1" + regenerator-runtime "^0.13.3" + +joi@^17.4.2: + version "17.6.0" + resolved "https://registry.npmmirror.com/joi/-/joi-17.6.0.tgz#0bb54f2f006c09a96e75ce687957bd04290054b2" + integrity sha512-OX5dG6DTbcr/kbMFj0KGYxuew69HPcAE3K/sZpEV2nP6e/j/C0HV+HNiBPCASxdx5T7DMoa0s8UeHWMnb6n2zw== + dependencies: + "@hapi/hoek" "^9.0.0" + "@hapi/topo" "^5.0.0" + "@sideway/address" "^4.1.3" + "@sideway/formula" "^3.0.0" + "@sideway/pinpoint" "^2.0.0" + +jpeg-js@0.4.2: + version "0.4.2" + resolved "https://registry.npmmirror.com/jpeg-js/-/jpeg-js-0.4.2.tgz#8b345b1ae4abde64c2da2fe67ea216a114ac279d" + integrity sha512-+az2gi/hvex7eLTMTlbRLOhH6P6WFdk2ITI8HJsaH2VqYO0I594zXSYEP+tf4FW+8Cy68ScDXoAsQdyQanv3sw== + +js-base64@^2.1.8: + version "2.6.4" + resolved "https://registry.npmmirror.com/js-base64/-/js-base64-2.6.4.tgz#f4e686c5de1ea1f867dbcad3d46d969428df98c4" + integrity sha512-pZe//GGmwJndub7ZghVHz7vjb2LgC1m8B07Au3eYqeqv9emhESByMXxaEgkUkEqJe87oBbSniGYoQNIBklc7IQ== + +"js-tokens@^3.0.0 || ^4.0.0", js-tokens@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" + integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== + +js-yaml@^3.13.1, js-yaml@^3.14.1: + version "3.14.1" + resolved "https://registry.npmmirror.com/js-yaml/-/js-yaml-3.14.1.tgz#dae812fdb3825fa306609a8717383c50c36a0537" + integrity sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g== + dependencies: + argparse "^1.0.7" + esprima "^4.0.0" + +jsbn@~0.1.0: + version "0.1.1" + resolved "https://registry.npmmirror.com/jsbn/-/jsbn-0.1.1.tgz#a5e654c2e5a2deb5f201d96cefbca80c0ef2f513" + integrity sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg== + +jsesc@^2.5.1: + version "2.5.2" + resolved "https://registry.npmmirror.com/jsesc/-/jsesc-2.5.2.tgz#80564d2e483dacf6e8ef209650a67df3f0c283a4" + integrity sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA== + +jsesc@~0.5.0: + version "0.5.0" + resolved "https://registry.npmmirror.com/jsesc/-/jsesc-0.5.0.tgz#e7dee66e35d6fc16f710fe91d5cf69f70f08911d" + integrity sha512-uZz5UnB7u4T9LvwmFqXii7pZSouaRPorGs5who1Ip7VO0wxanFvBL7GkM6dTHlgX+jhBApRetaWpnDabOeTcnA== + +json-buffer@3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/json-buffer/-/json-buffer-3.0.0.tgz#5b1f397afc75d677bde8bcfc0e47e1f9a3d9a898" + integrity sha512-CuUqjv0FUZIdXkHPI8MezCnFCdaTAacej1TZYulLoAg1h/PhwkdXFN4V/gzY4g+fMBCOV2xF+rp7t2XD2ns/NQ== + +json-buffer@3.0.1, json-buffer@~3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/json-buffer/-/json-buffer-3.0.1.tgz#9338802a30d3b6605fbe0613e094008ca8c05a13" + integrity sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ== + +json-loader@^0.5.7: + version "0.5.7" + resolved "https://registry.npmmirror.com/json-loader/-/json-loader-0.5.7.tgz#dca14a70235ff82f0ac9a3abeb60d337a365185d" + integrity sha512-QLPs8Dj7lnf3e3QYS1zkCo+4ZwqOiF9d/nZnYozTISxXWCfNs9yuky5rJw4/W34s7POaNlbZmQGaB5NiXCbP4w== + +json-parse-even-better-errors@^2.3.0, json-parse-even-better-errors@^2.3.1: + version "2.3.1" + resolved "https://registry.npmmirror.com/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz#7c47805a94319928e05777405dc12e1f7a4ee02d" + integrity sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w== + +json-schema-traverse@^0.4.1: + version "0.4.1" + resolved "https://registry.npmmirror.com/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz#69f6a87d9513ab8bb8fe63bdb0979c448e684660" + integrity sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg== + +json-schema-traverse@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz#ae7bcb3656ab77a73ba5c49bf654f38e6b6860e2" + integrity sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug== + +json-schema@0.4.0: + version "0.4.0" + resolved "https://registry.npmmirror.com/json-schema/-/json-schema-0.4.0.tgz#f7de4cf6efab838ebaeb3236474cbba5a1930ab5" + integrity sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA== + +json-stable-stringify-without-jsonify@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz#9db7b59496ad3f3cfef30a75142d2d930ad72651" + integrity sha512-Bdboy+l7tA3OGW6FjyFHWkP5LuByj1Tk33Ljyq0axyzdk9//JSi2u3fP1QSmd1KNwq6VOKYGlAu87CisVir6Pw== + +json-stringify-safe@~5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz#1296a2d58fd45f19a0f6ce01d65701e2c735b6eb" + integrity sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA== + +json2mq@^0.2.0: + version "0.2.0" + resolved "https://registry.npmmirror.com/json2mq/-/json2mq-0.2.0.tgz#b637bd3ba9eabe122c83e9720483aeb10d2c904a" + integrity sha512-SzoRg7ux5DWTII9J2qkrZrqV1gt+rTaoufMxEzXbS26Uid0NwaJd123HcoB80TgubEppxxIGdNxCx50fEoEWQA== + dependencies: + string-convert "^0.2.0" + +json5@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/json5/-/json5-1.0.1.tgz#779fb0018604fa854eacbf6252180d83543e3dbe" + integrity sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow== + dependencies: + minimist "^1.2.0" + +json5@^2.1.2, json5@^2.2.0, json5@^2.2.1: + version "2.2.1" + resolved "https://registry.npmmirror.com/json5/-/json5-2.2.1.tgz#655d50ed1e6f95ad1a3caababd2b0efda10b395c" + integrity sha512-1hqLFMSrGHRHxav9q9gNjJ5EXznIxGVO09xQRrwplcS8qs28pZ8s8hupZAmqDwZUmVZ2Qb2jnyPOWcDH8m8dlA== + +jsonfile@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/jsonfile/-/jsonfile-4.0.0.tgz#8771aae0799b64076b76640fca058f9c10e33ecb" + integrity sha512-m6F1R3z8jjlf2imQHS2Qez5sjKWQzbuuhuJ/FKYFRZvPE3PuHcSMVZzfsLhGVOkfd20obL5SWEBew5ShlquNxg== + optionalDependencies: + graceful-fs "^4.1.6" + +jsonfile@^6.0.1: + version "6.1.0" + resolved "https://registry.npmmirror.com/jsonfile/-/jsonfile-6.1.0.tgz#bc55b2634793c679ec6403094eb13698a6ec0aae" + integrity sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ== + dependencies: + universalify "^2.0.0" + optionalDependencies: + graceful-fs "^4.1.6" + +jsprim@^1.2.2: + version "1.4.2" + resolved "https://registry.npmmirror.com/jsprim/-/jsprim-1.4.2.tgz#712c65533a15c878ba59e9ed5f0e26d5b77c5feb" + integrity sha512-P2bSOMAc/ciLz6DzgjVlGJP9+BrJWu5UDGK70C2iweC5QBIeFf0ZXRvGjEj2uYgrY2MkAAhsSWHDWlFtEroZWw== + dependencies: + assert-plus "1.0.0" + extsprintf "1.3.0" + json-schema "0.4.0" + verror "1.10.0" + +"jsx-ast-utils@^2.4.1 || ^3.0.0", jsx-ast-utils@^3.2.1: + version "3.3.0" + resolved "https://registry.npmmirror.com/jsx-ast-utils/-/jsx-ast-utils-3.3.0.tgz#e624f259143b9062c92b6413ff92a164c80d3ccb" + integrity sha512-XzO9luP6L0xkxwhIJMTJQpZo/eeN60K08jHdexfD569AGxeNug6UketeHXEhROoM8aR7EcUoOQmIhcJQjcuq8Q== + dependencies: + array-includes "^3.1.4" + object.assign "^4.1.2" + +kebab-hash@^0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/kebab-hash/-/kebab-hash-0.1.2.tgz#dfb7949ba34d8e70114ea7d83e266e5e2a4abaac" + integrity sha512-BTZpq3xgISmQmAVzkISy4eUutsUA7s4IEFlCwOBJjvSFOwyR7I+fza+tBc/rzYWK/NrmFHjfU1IhO3lu29Ib/w== + dependencies: + lodash.kebabcase "^4.1.1" + +keyv@^3.0.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/keyv/-/keyv-3.1.0.tgz#ecc228486f69991e49e9476485a5be1e8fc5c4d9" + integrity sha512-9ykJ/46SN/9KPM/sichzQ7OvXyGDYKGTaDlKMGCAlg2UK8KRy4jb0d8sFc+0Tt0YYnThq8X2RZgCg74RPxgcVA== + dependencies: + json-buffer "3.0.0" + +keyv@^4.0.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/keyv/-/keyv-4.3.0.tgz#b4352e0e4fe7c94111947d6738a6d3fe7903027c" + integrity sha512-C30Un9+63J0CsR7Wka5quXKqYZsT6dcRQ2aOwGcSc3RiQ4HGWpTAHlCA+puNfw2jA/s11EsxA1nCXgZRuRKMQQ== + dependencies: + compress-brotli "^1.3.8" + json-buffer "3.0.1" + +kind-of@^6.0.0, kind-of@^6.0.2, kind-of@^6.0.3: + version "6.0.3" + resolved "https://registry.npmmirror.com/kind-of/-/kind-of-6.0.3.tgz#07c05034a6c349fa06e24fa35aa76db4580ce4dd" + integrity sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw== + +kleur@^3.0.3: + version "3.0.3" + resolved "https://registry.npmmirror.com/kleur/-/kleur-3.0.3.tgz#a79c9ecc86ee1ce3fa6206d1216c501f147fc07e" + integrity sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w== + +klona@^2.0.4: + version "2.0.5" + resolved "https://registry.npmmirror.com/klona/-/klona-2.0.5.tgz#d166574d90076395d9963aa7a928fabb8d76afbc" + integrity sha512-pJiBpiXMbt7dkzXe8Ghj/u4FfXOOa98fPW+bihOJ4SjnoijweJrNThJfd3ifXpXhREjpoF2mZVH1GfS9LV3kHQ== + +language-subtag-registry@~0.3.2: + version "0.3.21" + resolved "https://registry.npmmirror.com/language-subtag-registry/-/language-subtag-registry-0.3.21.tgz#04ac218bea46f04cb039084602c6da9e788dd45a" + integrity sha512-L0IqwlIXjilBVVYKFT37X9Ih11Um5NEl9cbJIuU/SwP/zEEAbBPOnEeeuxVMf45ydWQRDQN3Nqc96OgbH1K+Pg== + +language-tags@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/language-tags/-/language-tags-1.0.5.tgz#d321dbc4da30ba8bf3024e040fa5c14661f9193a" + integrity sha512-qJhlO9cGXi6hBGKoxEG/sKZDAHD5Hnu9Hs4WbOY3pCWXDhw0N8x1NenNzm2EnNLkLkk7J2SdxAkDSbb6ftT+UQ== + dependencies: + language-subtag-registry "~0.3.2" + +latest-version@5.1.0, latest-version@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/latest-version/-/latest-version-5.1.0.tgz#119dfe908fe38d15dfa43ecd13fa12ec8832face" + integrity sha512-weT+r0kTkRQdCdYCNtkMwWXQTMEswKrFBkm4ckQOMVhhqhIMI1UT2hMj+1iigIhgSZm5gTmrRXBNoGUgaTY1xA== + dependencies: + package-json "^6.3.0" + +levn@^0.4.1: + version "0.4.1" + resolved "https://registry.npmmirror.com/levn/-/levn-0.4.1.tgz#ae4562c007473b932a6200d403268dd2fffc6ade" + integrity sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ== + dependencies: + prelude-ls "^1.2.1" + type-check "~0.4.0" + +lilconfig@^2.0.3, lilconfig@^2.0.5: + version "2.0.5" + resolved "https://registry.npmmirror.com/lilconfig/-/lilconfig-2.0.5.tgz#19e57fd06ccc3848fd1891655b5a447092225b25" + integrity sha512-xaYmXZtTHPAw5m+xLN8ab9C+3a8YmV3asNSPOATITbtwrfbwaLJj8h66H1WMIpALCkqsIzK3h7oQ+PdX+LQ9Eg== + +lines-and-columns@^1.1.6: + version "1.2.4" + resolved "https://registry.npmmirror.com/lines-and-columns/-/lines-and-columns-1.2.4.tgz#eca284f75d2965079309dc0ad9255abb2ebc1632" + integrity sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg== + +lmdb-darwin-arm64@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-darwin-arm64/-/lmdb-darwin-arm64-2.3.10.tgz#4e20f75770eeedc60af3d4630975fd105a89ffe8" + integrity sha512-LVXbH2MYu7/ZuQ8+P9rv+SwNyBKltxo7vHAGJS94HWyfwnCbKEYER9PImBvNBwzvgtaYk6x0RMX3oor6e6KdDQ== + +lmdb-darwin-x64@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-darwin-x64/-/lmdb-darwin-x64-2.3.10.tgz#e53637a6735488eaa15feb7c0e9da142015b9476" + integrity sha512-gAc/1b/FZOb9yVOT+o0huA+hdW82oxLo5r22dFTLoRUFG1JMzxdTjmnW6ONVOHdqC9a5bt3vBCEY3jmXNqV26A== + +lmdb-linux-arm64@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-linux-arm64/-/lmdb-linux-arm64-2.3.10.tgz#ac7db8bdfe0e9dbf2be1cc3362d6f2b79e2a9722" + integrity sha512-Ihr8mdICTK3jA4GXHxrXGK2oekn0mY6zuDSXQDNtyRSH19j3D2Y04A7SEI9S0EP/t5sjKSudYgZbiHDxRCsI5A== + +lmdb-linux-arm@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-linux-arm/-/lmdb-linux-arm-2.3.10.tgz#74235418bbe7bf41e8ea5c9d52365c4ff5ca4b49" + integrity sha512-Rb8+4JjsThuEcJ7GLLwFkCFnoiwv/3hAAbELWITz70buQFF+dCZvCWWgEgmDTxwn5r+wIkdUjmFv4dqqiKQFmQ== + +lmdb-linux-x64@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-linux-x64/-/lmdb-linux-x64-2.3.10.tgz#d790b95061d03c5c99a57b3ad5126f7723c60a2f" + integrity sha512-E3l3pDiCA9uvnLf+t3qkmBGRO01dp1EHD0x0g0iRnfpAhV7wYbayJGfG93BUt22Tj3fnq4HDo4dQ6ZWaDI1nuw== + +lmdb-win32-x64@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb-win32-x64/-/lmdb-win32-x64-2.3.10.tgz#bff73d12d94084343c569b16069d8d38626eb2d6" + integrity sha512-gspWk34tDANhjn+brdqZstJMptGiwj4qFNVg0Zey9ds+BUlif+Lgf5szrfOVzZ8gVRkk1Lgbz7i78+V7YK7SCA== + +lmdb@2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/lmdb/-/lmdb-2.3.10.tgz#640fc60815846babcbe088d7f8ed0a51da857f6a" + integrity sha512-GtH+nStn9V59CfYeQ5ddx6YTfuFCmu86UJojIjJAweG+/Fm0PDknuk3ovgYDtY/foMeMdZa8/P7oSljW/d5UPw== + dependencies: + msgpackr "^1.5.4" + nan "^2.14.2" + node-addon-api "^4.3.0" + node-gyp-build-optional-packages "^4.3.2" + ordered-binary "^1.2.4" + weak-lru-cache "^1.2.2" + optionalDependencies: + lmdb-darwin-arm64 "2.3.10" + lmdb-darwin-x64 "2.3.10" + lmdb-linux-arm "2.3.10" + lmdb-linux-arm64 "2.3.10" + lmdb-linux-x64 "2.3.10" + lmdb-win32-x64 "2.3.10" + +lmdb@2.5.2: + version "2.5.2" + resolved "https://registry.npmmirror.com/lmdb/-/lmdb-2.5.2.tgz#37e28a9fb43405f4dc48c44cec0e13a14c4a6ff1" + integrity sha512-V5V5Xa2Hp9i2XsbDALkBTeHXnBXh/lEmk9p22zdr7jtuOIY9TGhjK6vAvTpOOx9IKU4hJkRWZxn/HsvR1ELLtA== + dependencies: + msgpackr "^1.5.4" + node-addon-api "^4.3.0" + node-gyp-build-optional-packages "5.0.3" + ordered-binary "^1.2.4" + weak-lru-cache "^1.2.2" + optionalDependencies: + "@lmdb/lmdb-darwin-arm64" "2.5.2" + "@lmdb/lmdb-darwin-x64" "2.5.2" + "@lmdb/lmdb-linux-arm" "2.5.2" + "@lmdb/lmdb-linux-arm64" "2.5.2" + "@lmdb/lmdb-linux-x64" "2.5.2" + "@lmdb/lmdb-win32-x64" "2.5.2" + +load-bmfont@^1.3.1, load-bmfont@^1.4.0: + version "1.4.1" + resolved "https://registry.npmmirror.com/load-bmfont/-/load-bmfont-1.4.1.tgz#c0f5f4711a1e2ccff725a7b6078087ccfcddd3e9" + integrity sha512-8UyQoYmdRDy81Brz6aLAUhfZLwr5zV0L3taTQ4hju7m6biuwiWiJXjPhBJxbUQJA8PrkvJ/7Enqmwk2sM14soA== + dependencies: + buffer-equal "0.0.1" + mime "^1.3.4" + parse-bmfont-ascii "^1.0.3" + parse-bmfont-binary "^1.0.5" + parse-bmfont-xml "^1.1.4" + phin "^2.9.1" + xhr "^2.0.1" + xtend "^4.0.0" + +loader-runner@^4.2.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/loader-runner/-/loader-runner-4.3.0.tgz#c1b4a163b99f614830353b16755e7149ac2314e1" + integrity sha512-3R/1M+yS3j5ou80Me59j7F9IMs4PXs3VqRrm0TU3AbKPxlmpoY1TNscJV/oGJXo8qCatFGTfDbY6W6ipGOYXfg== + +loader-utils@1.2.3: + version "1.2.3" + resolved "https://registry.npmmirror.com/loader-utils/-/loader-utils-1.2.3.tgz#1ff5dc6911c9f0a062531a4c04b609406108c2c7" + integrity sha512-fkpz8ejdnEMG3s37wGL07iSBDg99O9D5yflE9RGNH3hRdx9SOwYfnGYdZOUIZitN8E+E2vkq3MUMYMvPYl5ZZA== + dependencies: + big.js "^5.2.2" + emojis-list "^2.0.0" + json5 "^1.0.1" + +loader-utils@^1.4.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/loader-utils/-/loader-utils-1.4.0.tgz#c579b5e34cb34b1a74edc6c1fb36bfa371d5a613" + integrity sha512-qH0WSMBtn/oHuwjy/NucEgbx5dbxxnxup9s4PVXJUDHZBQY+s0NWA9rJf53RBnQZxfch7euUui7hpoAPvALZdA== + dependencies: + big.js "^5.2.2" + emojis-list "^3.0.0" + json5 "^1.0.1" + +loader-utils@^2.0.0: + version "2.0.2" + resolved "https://registry.npmmirror.com/loader-utils/-/loader-utils-2.0.2.tgz#d6e3b4fb81870721ae4e0868ab11dd638368c129" + integrity sha512-TM57VeHptv569d/GKh6TAYdzKblwDNiumOdkFnejjD0XwTH87K90w3O7AiJRqdQoXygvi1VQTJTLGhJl7WqA7A== + dependencies: + big.js "^5.2.2" + emojis-list "^3.0.0" + json5 "^2.1.2" + +loader-utils@^3.2.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/loader-utils/-/loader-utils-3.2.0.tgz#bcecc51a7898bee7473d4bc6b845b23af8304d4f" + integrity sha512-HVl9ZqccQihZ7JM85dco1MvO9G+ONvxoGa9rkhzFsneGLKSUg1gJf9bWzhRhcvm2qChhWpebQhP44qxjKIUCaQ== + +locate-path@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/locate-path/-/locate-path-2.0.0.tgz#2b568b265eec944c6d9c0de9c3dbbbca0354cd8e" + integrity sha512-NCI2kiDkyR7VeEKm27Kda/iQHyKJe1Bu0FlTbYp3CqJu+9IFe9bLyAjMxf5ZDDbEg+iMPzB5zYyUTSm8wVTKmA== + dependencies: + p-locate "^2.0.0" + path-exists "^3.0.0" + +locate-path@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/locate-path/-/locate-path-3.0.0.tgz#dbec3b3ab759758071b58fe59fc41871af21400e" + integrity sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A== + dependencies: + p-locate "^3.0.0" + path-exists "^3.0.0" + +locate-path@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" + integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== + dependencies: + p-locate "^4.1.0" + +locate-path@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/locate-path/-/locate-path-6.0.0.tgz#55321eb309febbc59c4801d931a72452a681d286" + integrity sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw== + dependencies: + p-locate "^5.0.0" + +lock@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/lock/-/lock-1.1.0.tgz#53157499d1653b136ca66451071fca615703fa55" + integrity sha512-NZQIJJL5Rb9lMJ0Yl1JoVr9GSdo4HTPsUEWsSFzB8dE8DSoiLCVavWZPi7Rnlv/o73u6I24S/XYc/NmG4l8EKA== + +lockfile@^1.0: + version "1.0.4" + resolved "https://registry.npmmirror.com/lockfile/-/lockfile-1.0.4.tgz#07f819d25ae48f87e538e6578b6964a4981a5609" + integrity sha512-cvbTwETRfsFh4nHsL1eGWapU1XFi5Ot9E85sWAwia7Y7EgB7vfqcZhTKZ+l7hCGxSPoushMv5GKhT5PdLv03WA== + dependencies: + signal-exit "^3.0.2" + +lodash._reinterpolate@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/lodash._reinterpolate/-/lodash._reinterpolate-3.0.0.tgz#0ccf2d89166af03b3663c796538b75ac6e114d9d" + integrity sha512-xYHt68QRoYGjeeM/XOE1uJtvXQAgvszfBhjV4yvsQH0u2i9I6cI6c6/eG4Hh3UAOVn0y/xAXwmTzEay49Q//HA== + +lodash.clonedeep@4.5.0: + version "4.5.0" + resolved "https://registry.npmmirror.com/lodash.clonedeep/-/lodash.clonedeep-4.5.0.tgz#e23f3f9c4f8fbdde872529c1071857a086e5ccef" + integrity sha512-H5ZhCF25riFd9uB5UCkVKo61m3S/xZk1x4wA6yp/L3RFP6Z/eHH1ymQcGLo7J3GMPfm0V/7m1tryHuGVxpqEBQ== + +lodash.curry@^4.0.1: + version "4.1.1" + resolved "https://registry.npmmirror.com/lodash.curry/-/lodash.curry-4.1.1.tgz#248e36072ede906501d75966200a86dab8b23170" + integrity sha512-/u14pXGviLaweY5JI0IUzgzF2J6Ne8INyzAZjImcryjgkZ+ebruBxy2/JaOOkTqScddcYtakjhSaeemV8lR0tA== + +lodash.debounce@^4.0.6, lodash.debounce@^4.0.8: + version "4.0.8" + resolved "https://registry.npmmirror.com/lodash.debounce/-/lodash.debounce-4.0.8.tgz#82d79bff30a67c4005ffd5e2515300ad9ca4d7af" + integrity sha512-FT1yDzDYEoYWhnSGnpE/4Kj1fLZkDFyqRb7fNt6FdYOSxlUWAtp42Eh6Wb0rGIv/m9Bgo7x4GhQbm5Ys4SG5ow== + +lodash.deburr@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/lodash.deburr/-/lodash.deburr-4.1.0.tgz#ddb1bbb3ef07458c0177ba07de14422cb033ff9b" + integrity sha512-m/M1U1f3ddMCs6Hq2tAsYThTBDaAKFDX3dwDo97GEYzamXi9SqUpjWi/Rrj/gf3X2n8ktwgZrlP1z6E3v/IExQ== + +lodash.every@^4.6.0: + version "4.6.0" + resolved "https://registry.npmmirror.com/lodash.every/-/lodash.every-4.6.0.tgz#eb89984bebc4364279bb3aefbbd1ca19bfa6c6a7" + integrity sha512-isF82d+65/sNvQ3aaQAW7LLHnnTxSN/2fm4rhYyuufLzA4VtHz6y6S5vFwe6PQVr2xdqUOyxBbTNKDpnmeu50w== + +lodash.flatten@^4.4.0: + version "4.4.0" + resolved "https://registry.npmmirror.com/lodash.flatten/-/lodash.flatten-4.4.0.tgz#f31c22225a9632d2bbf8e4addbef240aa765a61f" + integrity sha512-C5N2Z3DgnnKr0LOpv/hKCgKdb7ZZwafIrsesve6lmzvZIRZRGaZ/l6Q8+2W7NaT+ZwO3fFlSCzCzrDCFdJfZ4g== + +lodash.flattendeep@^4.4.0: + version "4.4.0" + resolved "https://registry.npmmirror.com/lodash.flattendeep/-/lodash.flattendeep-4.4.0.tgz#fb030917f86a3134e5bc9bec0d69e0013ddfedb2" + integrity sha512-uHaJFihxmJcEX3kT4I23ABqKKalJ/zDrDg0lsFtc1h+3uw49SIJ5beyhx5ExVRti3AvKoOJngIj7xz3oylPdWQ== + +lodash.flow@^3.3.0: + version "3.5.0" + resolved "https://registry.npmmirror.com/lodash.flow/-/lodash.flow-3.5.0.tgz#87bf40292b8cf83e4e8ce1a3ae4209e20071675a" + integrity sha512-ff3BX/tSioo+XojX4MOsOMhJw0nZoUEF011LX8g8d3gvjVbxd89cCio4BCXronjxcTUIJUoqKEUA+n4CqvvRPw== + +lodash.foreach@^4.5.0: + version "4.5.0" + resolved "https://registry.npmmirror.com/lodash.foreach/-/lodash.foreach-4.5.0.tgz#1a6a35eace401280c7f06dddec35165ab27e3e53" + integrity sha512-aEXTF4d+m05rVOAUG3z4vZZ4xVexLKZGF0lIxuHZ1Hplpk/3B6Z1+/ICICYRLm7c41Z2xiejbkCkJoTlypoXhQ== + +lodash.get@^4, lodash.get@^4.0: + version "4.4.2" + resolved "https://registry.npmmirror.com/lodash.get/-/lodash.get-4.4.2.tgz#2d177f652fa31e939b4438d5341499dfa3825e99" + integrity sha512-z+Uw/vLuy6gQe8cfaFWD7p0wVv8fJl3mbzXh33RS+0oW2wvUqiRXiQ69gLWSLpgB5/6sU+r6BlQR0MBILadqTQ== + +lodash.has@^4.0: + version "4.5.2" + resolved "https://registry.npmmirror.com/lodash.has/-/lodash.has-4.5.2.tgz#d19f4dc1095058cccbe2b0cdf4ee0fe4aa37c862" + integrity sha512-rnYUdIo6xRCJnQmbVFEwcxF144erlD+M3YcJUVesflU9paQaE8p+fJDcIQrlMYbxoANFL+AB9hZrzSBBk5PL+g== + +lodash.kebabcase@^4.1.1: + version "4.1.1" + resolved "https://registry.npmmirror.com/lodash.kebabcase/-/lodash.kebabcase-4.1.1.tgz#8489b1cb0d29ff88195cceca448ff6d6cc295c36" + integrity sha512-N8XRTIMMqqDgSy4VLKPnJ/+hpGZN+PHQiJnSenYqPaVV/NCqEogTnAdZLQiGKhxX+JCs8waWq2t1XHWKOmlY8g== + +lodash.map@^4.6.0: + version "4.6.0" + resolved "https://registry.npmmirror.com/lodash.map/-/lodash.map-4.6.0.tgz#771ec7839e3473d9c4cde28b19394c3562f4f6d3" + integrity sha512-worNHGKLDetmcEYDvh2stPCrrQRkP20E4l0iIS7F8EvzMqBBi7ltvFN5m1HvTf1P7Jk1txKhvFcmYsCr8O2F1Q== + +lodash.maxby@^4.6.0: + version "4.6.0" + resolved "https://registry.npmmirror.com/lodash.maxby/-/lodash.maxby-4.6.0.tgz#082240068f3c7a227aa00a8380e4f38cf0786e3d" + integrity sha512-QfTqQTwzmKxLy7VZlbx2M/ipWv8DCQ2F5BI/MRxLharOQ5V78yMSuB+JE+EuUM22txYfj09R2Q7hUlEYj7KdNg== + +lodash.memoize@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/lodash.memoize/-/lodash.memoize-4.1.2.tgz#bcc6c49a42a2840ed997f323eada5ecd182e0bfe" + integrity sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag== + +lodash.merge@^4.6.2: + version "4.6.2" + resolved "https://registry.npmmirror.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" + integrity sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ== + +lodash.template@^4.4.0: + version "4.5.0" + resolved "https://registry.npmmirror.com/lodash.template/-/lodash.template-4.5.0.tgz#f976195cf3f347d0d5f52483569fe8031ccce8ab" + integrity sha512-84vYFxIkmidUiFxidA/KjjH9pAycqW+h980j7Fuz5qxRtO9pgB7MDFTdys1N7A5mcucRiDyEq4fusljItR1T/A== + dependencies: + lodash._reinterpolate "^3.0.0" + lodash.templatesettings "^4.0.0" + +lodash.templatesettings@^4.0.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/lodash.templatesettings/-/lodash.templatesettings-4.2.0.tgz#e481310f049d3cf6d47e912ad09313b154f0fb33" + integrity sha512-stgLz+i3Aa9mZgnjr/O+v9ruKZsPsndy7qPZOchbqk2cnTU1ZaldKK+v7m54WoKIyxiuMZTKT2H81F8BeAc3ZQ== + dependencies: + lodash._reinterpolate "^3.0.0" + +lodash.throttle@^4.0.1: + version "4.1.1" + resolved "https://registry.npmmirror.com/lodash.throttle/-/lodash.throttle-4.1.1.tgz#c23e91b710242ac70c37f1e1cda9274cc39bf2f4" + integrity sha512-wIkUCfVKpVsWo3JSZlc+8MB5it+2AN5W8J7YVMST30UrvcQNZ1Okbj+rbVniijTWE6FGYy4XJq/rHkas8qJMLQ== + +lodash.truncate@^4.4.2: + version "4.4.2" + resolved "https://registry.npmmirror.com/lodash.truncate/-/lodash.truncate-4.4.2.tgz#5a350da0b1113b837ecfffd5812cbe58d6eae193" + integrity sha512-jttmRe7bRse52OsWIMDLaXxWqRAmtIUccAQ3garviCqJjafXOfNMO0yMfNpdD6zbGaTU0P5Nz7e7gAT6cKmJRw== + +lodash.uniq@^4.5.0: + version "4.5.0" + resolved "https://registry.npmmirror.com/lodash.uniq/-/lodash.uniq-4.5.0.tgz#d0225373aeb652adc1bc82e4945339a842754773" + integrity sha512-xfBaXQd9ryd9dlSDvnvI0lvxfLJlYAZzXomUYzLKtUeOQvOP5piqAWuGtrhWeqaXK9hhoM/iyJc5AV+XfsX3HQ== + +lodash.without@^4.4.0: + version "4.4.0" + resolved "https://registry.npmmirror.com/lodash.without/-/lodash.without-4.4.0.tgz#3cd4574a00b67bae373a94b748772640507b7aac" + integrity sha512-M3MefBwfDhgKgINVuBJCO1YR3+gf6s9HNJsIiZ/Ru77Ws6uTb9eBuvrkpzO+9iLoAaRodGuq7tyrPCx+74QYGQ== + +lodash@4.17.21, lodash@^4.0.0, lodash@^4.17.10, lodash@^4.17.15, lodash@^4.17.19, lodash@^4.17.20, lodash@^4.17.21, lodash@^4.17.4, lodash@~4.17.0, lodash@~4.17.10: + version "4.17.21" + resolved "https://registry.npmmirror.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" + integrity sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg== + +longest-streak@^2.0.0: + version "2.0.4" + resolved "https://registry.npmmirror.com/longest-streak/-/longest-streak-2.0.4.tgz#b8599957da5b5dab64dee3fe316fa774597d90e4" + integrity sha512-vM6rUVCVUJJt33bnmHiZEvr7wPT78ztX7rojL+LW51bHtLh6HTjx84LA5W4+oa6aKEJA7jJu5LR6vQRBpA5DVg== + +loose-envify@^1.0.0, loose-envify@^1.1.0, loose-envify@^1.4.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/loose-envify/-/loose-envify-1.4.0.tgz#71ee51fa7be4caec1a63839f7e682d8132d30caf" + integrity sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q== + dependencies: + js-tokens "^3.0.0 || ^4.0.0" + +lower-case-first@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/lower-case-first/-/lower-case-first-2.0.2.tgz#64c2324a2250bf7c37c5901e76a5b5309301160b" + integrity sha512-EVm/rR94FJTZi3zefZ82fLWab+GX14LJN4HrWBcuo6Evmsl9hEfnqxgcHCKb9q+mNf6EVdsjx/qucYFIIB84pg== + dependencies: + tslib "^2.0.3" + +lower-case@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/lower-case/-/lower-case-2.0.2.tgz#6fa237c63dbdc4a82ca0fd882e4722dc5e634e28" + integrity sha512-7fm3l3NAF9WfN6W3JOmf5drwpVqX78JtoGJ3A6W0a6ZnldM41w2fV5D490psKFTpMds8TJse/eHLFFsNHHjHgg== + dependencies: + tslib "^2.0.3" + +lowercase-keys@^1.0.0, lowercase-keys@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/lowercase-keys/-/lowercase-keys-1.0.1.tgz#6f9e30b47084d971a7c820ff15a6c5167b74c26f" + integrity sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA== + +lowercase-keys@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/lowercase-keys/-/lowercase-keys-2.0.0.tgz#2603e78b7b4b0006cbca2fbcc8a3202558ac9479" + integrity sha512-tqNXrS78oMOE73NMxK4EMLQsQowWf8jKooH9g7xPavRT706R6bkQJ6DY2Te7QukaZsulxa30wQ7bk0pm4XiHmA== + +lru-cache@4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/lru-cache/-/lru-cache-4.0.0.tgz#b5cbf01556c16966febe54ceec0fb4dc90df6c28" + integrity sha512-WKhDkjlLwzE8jAQdQlsxLUQTPXLCKX/4cJk6s5AlRtJkDBk0IKH5O51bVDH61K9N4bhbbyvLM6EiOuE8ovApPA== + dependencies: + pseudomap "^1.0.1" + yallist "^2.0.0" + +lru-cache@^4.0.0: + version "4.1.5" + resolved "https://registry.npmmirror.com/lru-cache/-/lru-cache-4.1.5.tgz#8bbe50ea85bed59bc9e33dcab8235ee9bcf443cd" + integrity sha512-sWZlbEP2OsHNkXrMl5GYk/jKk70MBng6UU4YI/qGDYbgf6YbP4EvmqISbXCoJiRKs+1bSpFHVgQxvJ17F2li5g== + dependencies: + pseudomap "^1.0.2" + yallist "^2.1.2" + +lru-cache@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/lru-cache/-/lru-cache-6.0.0.tgz#6d6fe6570ebd96aaf90fcad1dafa3b2566db3a94" + integrity sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA== + dependencies: + yallist "^4.0.0" + +lru-queue@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/lru-queue/-/lru-queue-0.1.0.tgz#2738bd9f0d3cf4f84490c5736c48699ac632cda3" + integrity sha512-BpdYkt9EvGl8OfWHDQPISVpcl5xZthb+XPsbELj5AQXxIC8IriDZIQYjBJPEm5rS420sjZ0TLEzRcq5KdBhYrQ== + dependencies: + es5-ext "~0.10.2" + +make-dir@^3.0.0, make-dir@^3.0.2, make-dir@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" + integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== + dependencies: + semver "^6.0.0" + +make-error@^1, make-error@^1.1.1: + version "1.3.6" + resolved "https://registry.npmmirror.com/make-error/-/make-error-1.3.6.tgz#2eb2e37ea9b67c4891f684a1394799af484cf7a2" + integrity sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw== + +map-age-cleaner@^0.1.3: + version "0.1.3" + resolved "https://registry.npmmirror.com/map-age-cleaner/-/map-age-cleaner-0.1.3.tgz#7d583a7306434c055fe474b0f45078e6e1b4b92a" + integrity sha512-bJzx6nMoP6PDLPBFmg7+xRKeFZvFboMrGlxmNj9ClvX53KrmvM5bXFXEWjbz4cz1AFn+jWJ9z/DJSz7hrs0w3w== + dependencies: + p-defer "^1.0.0" + +map-cache@^0.2.0: + version "0.2.2" + resolved "https://registry.npmmirror.com/map-cache/-/map-cache-0.2.2.tgz#c32abd0bd6525d9b051645bb4f26ac5dc98a0dbf" + integrity sha512-8y/eV9QQZCiyn1SprXSrCmqJN0yNRATe+PO8ztwqrvrbdRLA3eYJF0yaR0YayLWkMbsQSKWS9N2gPcGEc4UsZg== + +map-obj@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/map-obj/-/map-obj-1.0.1.tgz#d933ceb9205d82bdcf4886f6742bdc2b4dea146d" + integrity sha512-7N/q3lyZ+LVCp7PzuxrJr4KMbBE2hW7BT7YNia330OFxIf4d3r5zVpicP2650l7CPN6RM9zOJRl3NGpqSiw3Eg== + +map-obj@^4.0.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/map-obj/-/map-obj-4.3.0.tgz#9304f906e93faae70880da102a9f1df0ea8bb05a" + integrity sha512-hdN1wVrZbb29eBGiGjJbeP8JbKjq1urkHJ/LIP/NY48MZ1QVXUsQBV1G1zvYFHn1XE06cwjBsOI2K3Ulnj1YXQ== + +markdown-table@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/markdown-table/-/markdown-table-2.0.0.tgz#194a90ced26d31fe753d8b9434430214c011865b" + integrity sha512-Ezda85ToJUBhM6WGaG6veasyym+Tbs3cMAw/ZhOPqXiYsr0jgocBV3j3nx+4lk47plLlIqjwuTm/ywVI+zjJ/A== + dependencies: + repeat-string "^1.0.0" + +md5-file@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/md5-file/-/md5-file-5.0.0.tgz#e519f631feca9c39e7f9ea1780b63c4745012e20" + integrity sha512-xbEFXCYVWrSx/gEKS1VPlg84h/4L20znVIulKw6kMfmBUAZNAnF00eczz9ICMl+/hjQGo5KSXRxbL/47X3rmMw== + +mdast-util-definitions@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/mdast-util-definitions/-/mdast-util-definitions-4.0.0.tgz#c5c1a84db799173b4dcf7643cda999e440c24db2" + integrity sha512-k8AJ6aNnUkB7IE+5azR9h81O5EQ/cTDXtWdMq9Kk5KcEW/8ritU5CeLg/9HhOC++nALHBlaogJ5jz0Ybk3kPMQ== + dependencies: + unist-util-visit "^2.0.0" + +mdast-util-find-and-replace@^1.1.0: + version "1.1.1" + resolved "https://registry.npmmirror.com/mdast-util-find-and-replace/-/mdast-util-find-and-replace-1.1.1.tgz#b7db1e873f96f66588c321f1363069abf607d1b5" + integrity sha512-9cKl33Y21lyckGzpSmEQnIDjEfeeWelN5s1kUW1LwdB0Fkuq2u+4GdqcGEygYxJE8GVqCl0741bYXHgamfWAZA== + dependencies: + escape-string-regexp "^4.0.0" + unist-util-is "^4.0.0" + unist-util-visit-parents "^3.0.0" + +mdast-util-footnote@^0.1.0: + version "0.1.7" + resolved "https://registry.npmmirror.com/mdast-util-footnote/-/mdast-util-footnote-0.1.7.tgz#4b226caeab4613a3362c144c94af0fdd6f7e0ef0" + integrity sha512-QxNdO8qSxqbO2e3m09KwDKfWiLgqyCurdWTQ198NpbZ2hxntdc+VKS4fDJCmNWbAroUdYnSthu+XbZ8ovh8C3w== + dependencies: + mdast-util-to-markdown "^0.6.0" + micromark "~2.11.0" + +mdast-util-from-markdown@^0.8.0: + version "0.8.5" + resolved "https://registry.npmmirror.com/mdast-util-from-markdown/-/mdast-util-from-markdown-0.8.5.tgz#d1ef2ca42bc377ecb0463a987910dae89bd9a28c" + integrity sha512-2hkTXtYYnr+NubD/g6KGBS/0mFmBcifAsI0yIWRiRo0PjVs6SSOSOdtzbp6kSGnShDN6G5aWZpKQ2lWRy27mWQ== + dependencies: + "@types/mdast" "^3.0.0" + mdast-util-to-string "^2.0.0" + micromark "~2.11.0" + parse-entities "^2.0.0" + unist-util-stringify-position "^2.0.0" + +mdast-util-gfm-autolink-literal@^0.1.0: + version "0.1.3" + resolved "https://registry.npmmirror.com/mdast-util-gfm-autolink-literal/-/mdast-util-gfm-autolink-literal-0.1.3.tgz#9c4ff399c5ddd2ece40bd3b13e5447d84e385fb7" + integrity sha512-GjmLjWrXg1wqMIO9+ZsRik/s7PLwTaeCHVB7vRxUwLntZc8mzmTsLVr6HW1yLokcnhfURsn5zmSVdi3/xWWu1A== + dependencies: + ccount "^1.0.0" + mdast-util-find-and-replace "^1.1.0" + micromark "^2.11.3" + +mdast-util-gfm-strikethrough@^0.2.0: + version "0.2.3" + resolved "https://registry.npmmirror.com/mdast-util-gfm-strikethrough/-/mdast-util-gfm-strikethrough-0.2.3.tgz#45eea337b7fff0755a291844fbea79996c322890" + integrity sha512-5OQLXpt6qdbttcDG/UxYY7Yjj3e8P7X16LzvpX8pIQPYJ/C2Z1qFGMmcw+1PZMUM3Z8wt8NRfYTvCni93mgsgA== + dependencies: + mdast-util-to-markdown "^0.6.0" + +mdast-util-gfm-table@^0.1.0: + version "0.1.6" + resolved "https://registry.npmmirror.com/mdast-util-gfm-table/-/mdast-util-gfm-table-0.1.6.tgz#af05aeadc8e5ee004eeddfb324b2ad8c029b6ecf" + integrity sha512-j4yDxQ66AJSBwGkbpFEp9uG/LS1tZV3P33fN1gkyRB2LoRL+RR3f76m0HPHaby6F4Z5xr9Fv1URmATlRRUIpRQ== + dependencies: + markdown-table "^2.0.0" + mdast-util-to-markdown "~0.6.0" + +mdast-util-gfm-task-list-item@^0.1.0: + version "0.1.6" + resolved "https://registry.npmmirror.com/mdast-util-gfm-task-list-item/-/mdast-util-gfm-task-list-item-0.1.6.tgz#70c885e6b9f543ddd7e6b41f9703ee55b084af10" + integrity sha512-/d51FFIfPsSmCIRNp7E6pozM9z1GYPIkSy1urQ8s/o4TC22BZ7DqfHFWiqBD23bc7J3vV1Fc9O4QIHBlfuit8A== + dependencies: + mdast-util-to-markdown "~0.6.0" + +mdast-util-gfm@^0.1.0: + version "0.1.2" + resolved "https://registry.npmmirror.com/mdast-util-gfm/-/mdast-util-gfm-0.1.2.tgz#8ecddafe57d266540f6881f5c57ff19725bd351c" + integrity sha512-NNkhDx/qYcuOWB7xHUGWZYVXvjPFFd6afg6/e2g+SV4r9q5XUcCbV4Wfa3DLYIiD+xAEZc6K4MGaE/m0KDcPwQ== + dependencies: + mdast-util-gfm-autolink-literal "^0.1.0" + mdast-util-gfm-strikethrough "^0.2.0" + mdast-util-gfm-table "^0.1.0" + mdast-util-gfm-task-list-item "^0.1.0" + mdast-util-to-markdown "^0.6.1" + +mdast-util-to-hast@^10.2.0: + version "10.2.0" + resolved "https://registry.npmmirror.com/mdast-util-to-hast/-/mdast-util-to-hast-10.2.0.tgz#61875526a017d8857b71abc9333942700b2d3604" + integrity sha512-JoPBfJ3gBnHZ18icCwHR50orC9kNH81tiR1gs01D8Q5YpV6adHNO9nKNuFBCJQ941/32PT1a63UF/DitmS3amQ== + dependencies: + "@types/mdast" "^3.0.0" + "@types/unist" "^2.0.0" + mdast-util-definitions "^4.0.0" + mdurl "^1.0.0" + unist-builder "^2.0.0" + unist-util-generated "^1.0.0" + unist-util-position "^3.0.0" + unist-util-visit "^2.0.0" + +mdast-util-to-markdown@^0.6.0, mdast-util-to-markdown@^0.6.1, mdast-util-to-markdown@~0.6.0: + version "0.6.5" + resolved "https://registry.npmmirror.com/mdast-util-to-markdown/-/mdast-util-to-markdown-0.6.5.tgz#b33f67ca820d69e6cc527a93d4039249b504bebe" + integrity sha512-XeV9sDE7ZlOQvs45C9UKMtfTcctcaj/pGwH8YLbMHoMOXNNCn2LsqVQOqrF1+/NU8lKDAqozme9SCXWyo9oAcQ== + dependencies: + "@types/unist" "^2.0.0" + longest-streak "^2.0.0" + mdast-util-to-string "^2.0.0" + parse-entities "^2.0.0" + repeat-string "^1.0.0" + zwitch "^1.0.0" + +mdast-util-to-nlcst@^4.0.0: + version "4.0.1" + resolved "https://registry.npmmirror.com/mdast-util-to-nlcst/-/mdast-util-to-nlcst-4.0.1.tgz#ff8b5339c960b38209273fa8bf4dd7a9498f8636" + integrity sha512-Y4ffygj85MTt70STKnEquw6k73jYWJBaYcb4ITAKgSNokZF7fH8rEHZ1GsRY/JaxqUevMaEnsDmkVv5Z9uVRdg== + dependencies: + nlcst-to-string "^2.0.0" + repeat-string "^1.0.0" + unist-util-position "^3.0.0" + vfile-location "^3.1.0" + +mdast-util-to-string@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/mdast-util-to-string/-/mdast-util-to-string-2.0.0.tgz#b8cfe6a713e1091cb5b728fc48885a4767f8b97b" + integrity sha512-AW4DRS3QbBayY/jJmD8437V1Gombjf8RSOUCMFBuo5iHi58AGEgVCKQ+ezHkZZDpAQS75hcBMpLqjpJTjtUL7w== + +mdast-util-toc@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/mdast-util-toc/-/mdast-util-toc-5.1.0.tgz#3af0f9c9a764b993538af03f1f79f4e3cec22736" + integrity sha512-csimbRIVkiqc+PpFeKDGQ/Ck2N4f9FYH3zzBMMJzcxoKL8m+cM0n94xXm0I9eaxHnKdY9n145SGTdyJC7i273g== + dependencies: + "@types/mdast" "^3.0.3" + "@types/unist" "^2.0.3" + extend "^3.0.2" + github-slugger "^1.2.1" + mdast-util-to-string "^2.0.0" + unist-util-is "^4.0.0" + unist-util-visit "^2.0.0" + +mdn-data@2.0.14: + version "2.0.14" + resolved "https://registry.npmmirror.com/mdn-data/-/mdn-data-2.0.14.tgz#7113fc4281917d63ce29b43446f701e68c25ba50" + integrity sha512-dn6wd0uw5GsdswPFfsgMp5NSB0/aDe6fK94YJV/AJDYXL6HVLWBsxeq7js7Ad+mU2K9LAlwpk6kN2D5mwCPVow== + +mdn-data@2.0.4: + version "2.0.4" + resolved "https://registry.npmmirror.com/mdn-data/-/mdn-data-2.0.4.tgz#699b3c38ac6f1d728091a64650b65d388502fd5b" + integrity sha512-iV3XNKw06j5Q7mi6h+9vbx23Tv7JkjEVgKHW4pimwyDGWm0OIQntJJ+u1C6mg6mK1EaTv42XQ7w76yuzH7M2cA== + +mdurl@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/mdurl/-/mdurl-1.0.1.tgz#fe85b2ec75a59037f2adfec100fd6c601761152e" + integrity sha512-/sKlQJCBYVY9Ers9hqzKou4H6V5UWc/M59TH2dvkt+84itfnq7uFOMLpOiOS4ujvHP4etln18fmIxA5R5fll0g== + +meant@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/meant/-/meant-1.0.3.tgz#67769af9de1d158773e928ae82c456114903554c" + integrity sha512-88ZRGcNxAq4EH38cQ4D85PM57pikCwS8Z99EWHODxN7KBY+UuPiqzRTtZzS8KTXO/ywSWbdjjJST2Hly/EQxLw== + +media-typer@0.3.0: + version "0.3.0" + resolved "https://registry.npmmirror.com/media-typer/-/media-typer-0.3.0.tgz#8710d7af0aa626f8fffa1ce00168545263255748" + integrity sha512-dq+qelQ9akHpcOl/gUVRTxVIOkAJ1wR3QAvb4RsVjS8oVoFjDGTc679wJYmUmknUF5HwMLOgb5O+a3KxfWapPQ== + +mem@^8.1.1: + version "8.1.1" + resolved "https://registry.npmmirror.com/mem/-/mem-8.1.1.tgz#cf118b357c65ab7b7e0817bdf00c8062297c0122" + integrity sha512-qFCFUDs7U3b8mBDPyz5EToEKoAkgCzqquIgi9nkkR9bixxOVOre+09lbuH7+9Kn2NFpm56M3GUWVbU2hQgdACA== + dependencies: + map-age-cleaner "^0.1.3" + mimic-fn "^3.1.0" + +memfs@^3.1.2, memfs@^3.2.2: + version "3.4.4" + resolved "https://registry.npmmirror.com/memfs/-/memfs-3.4.4.tgz#e8973cd8060548916adcca58a248e7805c715e89" + integrity sha512-W4gHNUE++1oSJVn8Y68jPXi+mkx3fXR5ITE/Ubz6EQ3xRpCN5k2CQ4AUR8094Z7211F876TyoBACGsIveqgiGA== + dependencies: + fs-monkey "1.0.3" + +memoize-one@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/memoize-one/-/memoize-one-6.0.0.tgz#b2591b871ed82948aee4727dc6abceeeac8c1045" + integrity sha512-rkpe71W0N0c0Xz6QD0eJETuWAJGnJ9afsl1srmwPrI+yBCkge5EycXXbYRyvL29zZVUWQCY7InPRCv3GDXuZNw== + +memoizee@^0.4.15: + version "0.4.15" + resolved "https://registry.npmmirror.com/memoizee/-/memoizee-0.4.15.tgz#e6f3d2da863f318d02225391829a6c5956555b72" + integrity sha512-UBWmJpLZd5STPm7PMUlOw/TSy972M+z8gcyQ5veOnSDRREz/0bmpyTfKt3/51DhEBqCZQn1udM/5flcSPYhkdQ== + dependencies: + d "^1.0.1" + es5-ext "^0.10.53" + es6-weak-map "^2.0.3" + event-emitter "^0.3.5" + is-promise "^2.2.2" + lru-queue "^0.1.0" + next-tick "^1.1.0" + timers-ext "^0.1.7" + +meow@^9.0.0: + version "9.0.0" + resolved "https://registry.npmmirror.com/meow/-/meow-9.0.0.tgz#cd9510bc5cac9dee7d03c73ee1f9ad959f4ea364" + integrity sha512-+obSblOQmRhcyBt62furQqRAQpNyWXo8BuQ5bN7dG8wmwQ+vwHKp/rCFD4CrTP8CsDQD1sjoZ94K417XEUk8IQ== + dependencies: + "@types/minimist" "^1.2.0" + camelcase-keys "^6.2.2" + decamelize "^1.2.0" + decamelize-keys "^1.1.0" + hard-rejection "^2.1.0" + minimist-options "4.1.0" + normalize-package-data "^3.0.0" + read-pkg-up "^7.0.1" + redent "^3.0.0" + trim-newlines "^3.0.0" + type-fest "^0.18.0" + yargs-parser "^20.2.3" + +merge-descriptors@1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/merge-descriptors/-/merge-descriptors-1.0.1.tgz#b00aaa556dd8b44568150ec9d1b953f3f90cbb61" + integrity sha512-cCi6g3/Zr1iqQi6ySbseM1Xvooa98N0w31jzUYrXPX2xqObmFGHJ0tQ5u74H3mVh7wLouTseZyYIq39g8cNp1w== + +merge-stream@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" + integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== + +merge2@^1.3.0, merge2@^1.4.1: + version "1.4.1" + resolved "https://registry.npmmirror.com/merge2/-/merge2-1.4.1.tgz#4368892f885e907455a6fd7dc55c0c9d404990ae" + integrity sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg== + +meros@1.1.4: + version "1.1.4" + resolved "https://registry.npmmirror.com/meros/-/meros-1.1.4.tgz#c17994d3133db8b23807f62bec7f0cb276cfd948" + integrity sha512-E9ZXfK9iQfG9s73ars9qvvvbSIkJZF5yOo9j4tcwM5tN8mUKfj/EKN5PzOr3ZH0y5wL7dLAHw3RVEfpQV9Q7VQ== + +methods@~1.1.2: + version "1.1.2" + resolved "https://registry.npmmirror.com/methods/-/methods-1.1.2.tgz#5529a4d67654134edcc5266656835b0f851afcee" + integrity sha512-iclAHeNqNm68zFtnZ0e+1L2yUIdvzNoauKU4WBA3VvH/vPFieF7qfRlwUZU+DA9P9bPXIS90ulxoUoCH23sV2w== + +micromark-extension-footnote@^0.3.0: + version "0.3.2" + resolved "https://registry.npmmirror.com/micromark-extension-footnote/-/micromark-extension-footnote-0.3.2.tgz#129b74ef4920ce96719b2c06102ee7abb2b88a20" + integrity sha512-gr/BeIxbIWQoUm02cIfK7mdMZ/fbroRpLsck4kvFtjbzP4yi+OPVbnukTc/zy0i7spC2xYE/dbX1Sur8BEDJsQ== + dependencies: + micromark "~2.11.0" + +micromark-extension-gfm-autolink-literal@~0.5.0: + version "0.5.7" + resolved "https://registry.npmmirror.com/micromark-extension-gfm-autolink-literal/-/micromark-extension-gfm-autolink-literal-0.5.7.tgz#53866c1f0c7ef940ae7ca1f72c6faef8fed9f204" + integrity sha512-ePiDGH0/lhcngCe8FtH4ARFoxKTUelMp4L7Gg2pujYD5CSMb9PbblnyL+AAMud/SNMyusbS2XDSiPIRcQoNFAw== + dependencies: + micromark "~2.11.3" + +micromark-extension-gfm-strikethrough@~0.6.5: + version "0.6.5" + resolved "https://registry.npmmirror.com/micromark-extension-gfm-strikethrough/-/micromark-extension-gfm-strikethrough-0.6.5.tgz#96cb83356ff87bf31670eefb7ad7bba73e6514d1" + integrity sha512-PpOKlgokpQRwUesRwWEp+fHjGGkZEejj83k9gU5iXCbDG+XBA92BqnRKYJdfqfkrRcZRgGuPuXb7DaK/DmxOhw== + dependencies: + micromark "~2.11.0" + +micromark-extension-gfm-table@~0.4.0: + version "0.4.3" + resolved "https://registry.npmmirror.com/micromark-extension-gfm-table/-/micromark-extension-gfm-table-0.4.3.tgz#4d49f1ce0ca84996c853880b9446698947f1802b" + integrity sha512-hVGvESPq0fk6ALWtomcwmgLvH8ZSVpcPjzi0AjPclB9FsVRgMtGZkUcpE0zgjOCFAznKepF4z3hX8z6e3HODdA== + dependencies: + micromark "~2.11.0" + +micromark-extension-gfm-tagfilter@~0.3.0: + version "0.3.0" + resolved "https://registry.npmmirror.com/micromark-extension-gfm-tagfilter/-/micromark-extension-gfm-tagfilter-0.3.0.tgz#d9f26a65adee984c9ccdd7e182220493562841ad" + integrity sha512-9GU0xBatryXifL//FJH+tAZ6i240xQuFrSL7mYi8f4oZSbc+NvXjkrHemeYP0+L4ZUT+Ptz3b95zhUZnMtoi/Q== + +micromark-extension-gfm-task-list-item@~0.3.0: + version "0.3.3" + resolved "https://registry.npmmirror.com/micromark-extension-gfm-task-list-item/-/micromark-extension-gfm-task-list-item-0.3.3.tgz#d90c755f2533ed55a718129cee11257f136283b8" + integrity sha512-0zvM5iSLKrc/NQl84pZSjGo66aTGd57C1idmlWmE87lkMcXrTxg1uXa/nXomxJytoje9trP0NDLvw4bZ/Z/XCQ== + dependencies: + micromark "~2.11.0" + +micromark-extension-gfm@^0.3.0: + version "0.3.3" + resolved "https://registry.npmmirror.com/micromark-extension-gfm/-/micromark-extension-gfm-0.3.3.tgz#36d1a4c089ca8bdfd978c9bd2bf1a0cb24e2acfe" + integrity sha512-oVN4zv5/tAIA+l3GbMi7lWeYpJ14oQyJ3uEim20ktYFAcfX1x3LNlFGGlmrZHt7u9YlKExmyJdDGaTt6cMSR/A== + dependencies: + micromark "~2.11.0" + micromark-extension-gfm-autolink-literal "~0.5.0" + micromark-extension-gfm-strikethrough "~0.6.5" + micromark-extension-gfm-table "~0.4.0" + micromark-extension-gfm-tagfilter "~0.3.0" + micromark-extension-gfm-task-list-item "~0.3.0" + +micromark@^2.11.3, micromark@~2.11.0, micromark@~2.11.3: + version "2.11.4" + resolved "https://registry.npmmirror.com/micromark/-/micromark-2.11.4.tgz#d13436138eea826383e822449c9a5c50ee44665a" + integrity sha512-+WoovN/ppKolQOFIAajxi7Lu9kInbPxFuTBVEavFcL8eAfVstoc5MocPmqBeAdBOJV00uaVjegzH4+MA0DN/uA== + dependencies: + debug "^4.0.0" + parse-entities "^2.0.0" + +micromatch@^4.0.4, micromatch@^4.0.5: + version "4.0.5" + resolved "https://registry.npmmirror.com/micromatch/-/micromatch-4.0.5.tgz#bc8999a7cbbf77cdc89f132f6e467051b49090c6" + integrity sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA== + dependencies: + braces "^3.0.2" + picomatch "^2.3.1" + +mime-db@1.52.0, "mime-db@>= 1.43.0 < 2": + version "1.52.0" + resolved "https://registry.npmmirror.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70" + integrity sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg== + +mime-types@^2.1.12, mime-types@^2.1.27, mime-types@^2.1.30, mime-types@~2.1.19, mime-types@~2.1.24, mime-types@~2.1.34: + version "2.1.35" + resolved "https://registry.npmmirror.com/mime-types/-/mime-types-2.1.35.tgz#381a871b62a734450660ae3deee44813f70d959a" + integrity sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw== + dependencies: + mime-db "1.52.0" + +mime@1.6.0, mime@^1.3.4: + version "1.6.0" + resolved "https://registry.npmmirror.com/mime/-/mime-1.6.0.tgz#32cd9e5c64553bd58d19a568af452acff04981b1" + integrity sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg== + +mime@^2.4.4, mime@^2.5.2: + version "2.6.0" + resolved "https://registry.npmmirror.com/mime/-/mime-2.6.0.tgz#a2a682a95cd4d0cb1d6257e28f83da7e35800367" + integrity sha512-USPkMeET31rOMiarsBNIHZKLGgvKc/LrjofAnBlOttf5ajRvqiRA8QsenbcooctK6d6Ts6aqZXBA+XbkKthiQg== + +mime@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/mime/-/mime-3.0.0.tgz#b374550dca3a0c18443b0c950a6a58f1931cf7a7" + integrity sha512-jSCU7/VB1loIWBZe14aEYHU/+1UMEHoaO7qxCOVJOw9GgH72VAWppxNcjU+x9a2k3GSIBXNKxXQFqRvvZ7vr3A== + +mimic-fn@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" + integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== + +mimic-fn@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/mimic-fn/-/mimic-fn-3.1.0.tgz#65755145bbf3e36954b949c16450427451d5ca74" + integrity sha512-Ysbi9uYW9hFyfrThdDEQuykN4Ey6BuwPD2kpI5ES/nFTDn/98yxYNLZJcgUAKPT/mcrLLKaGzJR9YVxJrIdASQ== + +mimic-response@^1.0.0, mimic-response@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/mimic-response/-/mimic-response-1.0.1.tgz#4923538878eef42063cb8a3e3b0798781487ab1b" + integrity sha512-j5EctnkH7amfV/q5Hgmoal1g2QHFJRraOtmx0JpIqkxhBhI/lJSl1nMpQ45hVarwNETOoWEimndZ4QK0RHxuxQ== + +mimic-response@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/mimic-response/-/mimic-response-3.1.0.tgz#2d1d59af9c1b129815accc2c46a022a5ce1fa3c9" + integrity sha512-z0yWI+4FDrrweS8Zmt4Ej5HdJmky15+L2e6Wgn3+iK5fWzb6T3fhNFq2+MeTRb064c6Wr4N/wv0DzQTjNzHNGQ== + +min-document@^2.19.0: + version "2.19.0" + resolved "https://registry.npmmirror.com/min-document/-/min-document-2.19.0.tgz#7bd282e3f5842ed295bb748cdd9f1ffa2c824685" + integrity sha512-9Wy1B3m3f66bPPmU5hdA4DR4PB2OfDU/+GS3yAB7IQozE3tqXaVv2zOjgla7MEGSRv95+ILmOuvhLkOK6wJtCQ== + dependencies: + dom-walk "^0.1.0" + +min-indent@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/min-indent/-/min-indent-1.0.1.tgz#a63f681673b30571fbe8bc25686ae746eefa9869" + integrity sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg== + +mini-css-extract-plugin@1.6.2: + version "1.6.2" + resolved "https://registry.npmmirror.com/mini-css-extract-plugin/-/mini-css-extract-plugin-1.6.2.tgz#83172b4fd812f8fc4a09d6f6d16f924f53990ca8" + integrity sha512-WhDvO3SjGm40oV5y26GjMJYjd2UMqrLAGKy5YS2/3QKJy2F7jgynuHTir/tgUUOiNQu5saXHdc8reo7YuhhT4Q== + dependencies: + loader-utils "^2.0.0" + schema-utils "^3.0.0" + webpack-sources "^1.1.0" + +mini-svg-data-uri@^1.4.4: + version "1.4.4" + resolved "https://registry.npmmirror.com/mini-svg-data-uri/-/mini-svg-data-uri-1.4.4.tgz#8ab0aabcdf8c29ad5693ca595af19dd2ead09939" + integrity sha512-r9deDe9p5FJUPZAk3A59wGH7Ii9YrjjWw0jmw/liSbHl2CHiyXj6FcDXDu2K3TjVAXqiJdaw3xxwlZZr9E6nHg== + +minimatch@3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/minimatch/-/minimatch-3.0.4.tgz#5166e286457f03306064be5497e8dbb0c3d32083" + integrity sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA== + dependencies: + brace-expansion "^1.1.7" + +minimatch@^3.0.4, minimatch@^3.1.1, minimatch@^3.1.2: + version "3.1.2" + resolved "https://registry.npmmirror.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" + integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw== + dependencies: + brace-expansion "^1.1.7" + +minimatch@~3.0.2: + version "3.0.8" + resolved "https://registry.npmmirror.com/minimatch/-/minimatch-3.0.8.tgz#5e6a59bd11e2ab0de1cfb843eb2d82e546c321c1" + integrity sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q== + dependencies: + brace-expansion "^1.1.7" + +minimist-options@4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/minimist-options/-/minimist-options-4.1.0.tgz#c0655713c53a8a2ebd77ffa247d342c40f010619" + integrity sha512-Q4r8ghd80yhO/0j1O3B2BjweX3fiHg9cdOwjJd2J76Q135c+NDxGCqdYKQ1SKBuFfgWbAUzBfvYjPUEeNgqN1A== + dependencies: + arrify "^1.0.1" + is-plain-obj "^1.1.0" + kind-of "^6.0.3" + +minimist@^1.2.0, minimist@^1.2.3, minimist@^1.2.6, minimist@~1.2.5: + version "1.2.6" + resolved "https://registry.npmmirror.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44" + integrity sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q== + +minipass@^3.0.0: + version "3.1.6" + resolved "https://registry.npmmirror.com/minipass/-/minipass-3.1.6.tgz#3b8150aa688a711a1521af5e8779c1d3bb4f45ee" + integrity sha512-rty5kpw9/z8SX9dmxblFA6edItUmwJgMeYDZRrwlIVN27i8gysGbznJwUggw2V/FVqFSDdWy040ZPS811DYAqQ== + dependencies: + yallist "^4.0.0" + +minizlib@^2.1.1: + version "2.1.2" + resolved "https://registry.npmmirror.com/minizlib/-/minizlib-2.1.2.tgz#e90d3466ba209b932451508a11ce3d3632145931" + integrity sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg== + dependencies: + minipass "^3.0.0" + yallist "^4.0.0" + +mitt@^1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/mitt/-/mitt-1.2.0.tgz#cb24e6569c806e31bd4e3995787fe38a04fdf90d" + integrity sha512-r6lj77KlwqLhIUku9UWYes7KJtsczvolZkzp8hbaDPPaE24OmWl5s539Mytlj22siEQKosZ26qCBgda2PKwoJw== + +mkdirp-classic@^0.5.2, mkdirp-classic@^0.5.3: + version "0.5.3" + resolved "https://registry.npmmirror.com/mkdirp-classic/-/mkdirp-classic-0.5.3.tgz#fa10c9115cc6d8865be221ba47ee9bed78601113" + integrity sha512-gKLcREMhtuZRwRAfqP3RFW+TK4JqApVBtOIftVgjuABpAtpxhPGaDcfvbhNvD0B8iD1oUr/txX35NjcaY6Ns/A== + +mkdirp@^0.5.1, mkdirp@^0.5.4, mkdirp@~0.5.1: + version "0.5.6" + resolved "https://registry.npmmirror.com/mkdirp/-/mkdirp-0.5.6.tgz#7def03d2432dcae4ba1d611445c48396062255f6" + integrity sha512-FP+p8RB8OWpF3YZBCrP5gtADmtXApB5AMLn+vdyA+PyxCjrCs00mjyUozssO33cwDeT3wNGdLxJ5M//YqtHAJw== + dependencies: + minimist "^1.2.6" + +mkdirp@^1.0.3: + version "1.0.4" + resolved "https://registry.npmmirror.com/mkdirp/-/mkdirp-1.0.4.tgz#3eb5ed62622756d79a5f0e2a221dfebad75c2f7e" + integrity sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw== + +moment@^2.24.0, moment@^2.29.1, moment@^2.29.2: + version "2.29.3" + resolved "https://registry.npmmirror.com/moment/-/moment-2.29.3.tgz#edd47411c322413999f7a5940d526de183c031f3" + integrity sha512-c6YRvhEo//6T2Jz/vVtYzqBzwvPT95JBQ+smCytzf7c50oMZRsR/a4w88aD34I+/QVSfnoAnSBFPJHItlOMJVw== + +ms@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/ms/-/ms-2.0.0.tgz#5608aeadfc00be6c2901df5f9861788de0d597c8" + integrity sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A== + +ms@2.1.2: + version "2.1.2" + resolved "https://registry.npmmirror.com/ms/-/ms-2.1.2.tgz#d09d1f357b443f493382a8eb3ccd183872ae6009" + integrity sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w== + +ms@2.1.3, ms@^2.1.1: + version "2.1.3" + resolved "https://registry.npmmirror.com/ms/-/ms-2.1.3.tgz#574c8138ce1d2b5861f0b44579dbadd60c6615b2" + integrity sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA== + +msgpackr-extract@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/msgpackr-extract/-/msgpackr-extract-2.0.2.tgz#201a8d7ade47e99b3ba277c45736b00e195d4670" + integrity sha512-coskCeJG2KDny23zWeu+6tNy7BLnAiOGgiwzlgdm4oeSsTpqEJJPguHIuKZcCdB7tzhZbXNYSg6jZAXkZErkJA== + dependencies: + node-gyp-build-optional-packages "5.0.2" + optionalDependencies: + "@msgpackr-extract/msgpackr-extract-darwin-arm64" "2.0.2" + "@msgpackr-extract/msgpackr-extract-darwin-x64" "2.0.2" + "@msgpackr-extract/msgpackr-extract-linux-arm" "2.0.2" + "@msgpackr-extract/msgpackr-extract-linux-arm64" "2.0.2" + "@msgpackr-extract/msgpackr-extract-linux-x64" "2.0.2" + "@msgpackr-extract/msgpackr-extract-win32-x64" "2.0.2" + +msgpackr@^1.5.4: + version "1.6.1" + resolved "https://registry.npmmirror.com/msgpackr/-/msgpackr-1.6.1.tgz#4f3c94d6a5b819b838ffc736eddaf60eba436d20" + integrity sha512-Je+xBEfdjtvA4bKaOv8iRhjC8qX2oJwpYH4f7JrG4uMVJVmnmkAT4pjKdbztKprGj3iwjcxPzb5umVZ02Qq3tA== + optionalDependencies: + msgpackr-extract "^2.0.2" + +multer@^1.4.3: + version "1.4.4" + resolved "https://registry.npmmirror.com/multer/-/multer-1.4.4.tgz#e2bc6cac0df57a8832b858d7418ccaa8ebaf7d8c" + integrity sha512-2wY2+xD4udX612aMqMcB8Ws2Voq6NIUPEtD1be6m411T4uDH/VtL9i//xvcyFlTVfRdaBsk7hV5tgrGQqhuBiw== + dependencies: + append-field "^1.0.0" + busboy "^0.2.11" + concat-stream "^1.5.2" + mkdirp "^0.5.4" + object-assign "^4.1.1" + on-finished "^2.3.0" + type-is "^1.6.4" + xtend "^4.0.0" + +multipipe@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/multipipe/-/multipipe-1.0.2.tgz#cc13efd833c9cda99f224f868461b8e1a3fd939d" + integrity sha512-6uiC9OvY71vzSGX8lZvSqscE7ft9nPupJ8fMjrCNRAUy2LREUW42UL+V/NTrogr6rFgRydUrCX4ZitfpSNkSCQ== + dependencies: + duplexer2 "^0.1.2" + object-assign "^4.1.0" + +mute-stream@0.0.8, mute-stream@~0.0.4: + version "0.0.8" + resolved "https://registry.npmmirror.com/mute-stream/-/mute-stream-0.0.8.tgz#1630c42b2251ff81e2a283de96a5497ea92e5e0d" + integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA== + +nan@^2.13.2, nan@^2.14.2: + version "2.16.0" + resolved "https://registry.npmmirror.com/nan/-/nan-2.16.0.tgz#664f43e45460fb98faf00edca0bb0d7b8dce7916" + integrity sha512-UdAqHyFngu7TfQKsCBgAA6pWDkT8MAO7d0jyOecVhN5354xbLqdn8mV9Tat9gepAupm0bt2DbeaSC8vS52MuFA== + +nanoid@^3.3.4: + version "3.3.4" + resolved "https://registry.npmmirror.com/nanoid/-/nanoid-3.3.4.tgz#730b67e3cd09e2deacf03c027c81c9d9dbc5e8ab" + integrity sha512-MqBkQh/OHTS2egovRtLk45wEyNXwF+cokD+1YPf9u5VfJiRdAiRwB2froX5Co9Rh20xs4siNPm8naNotSD6RBw== + +napi-build-utils@^1.0.1: + version "1.0.2" + resolved "https://registry.npmmirror.com/napi-build-utils/-/napi-build-utils-1.0.2.tgz#b1fddc0b2c46e380a0b7a76f984dd47c41a13806" + integrity sha512-ONmRUqK7zj7DWX0D9ADe03wbwOBZxNAfF20PlGfCWQcD3+/MakShIHrMqx9YwPTfxDdF1zLeL+RGZiR9kGMLdg== + +native-url@^0.2.6: + version "0.2.6" + resolved "https://registry.npmmirror.com/native-url/-/native-url-0.2.6.tgz#ca1258f5ace169c716ff44eccbddb674e10399ae" + integrity sha512-k4bDC87WtgrdD362gZz6zoiXQrl40kYlBmpfmSjwRO1VU0V5ccwJTlxuE72F6m3V0vc1xOf6n3UCP9QyerRqmA== + dependencies: + querystring "^0.2.0" + +natural-compare@^1.4.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7" + integrity sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw== + +needle@^2.5.2: + version "2.9.1" + resolved "https://registry.npmmirror.com/needle/-/needle-2.9.1.tgz#22d1dffbe3490c2b83e301f7709b6736cd8f2684" + integrity sha512-6R9fqJ5Zcmf+uYaFgdIHmLwNldn5HbK8L5ybn7Uz+ylX/rnOsSp1AHcvQSrCaFN+qNM1wpymHqD7mVasEOlHGQ== + dependencies: + debug "^3.2.6" + iconv-lite "^0.4.4" + sax "^1.2.4" + +negotiator@0.6.3, negotiator@~0.6.2: + version "0.6.3" + resolved "https://registry.npmmirror.com/negotiator/-/negotiator-0.6.3.tgz#58e323a72fedc0d6f9cd4d31fe49f51479590ccd" + integrity sha512-+EUsqGPLsM+j/zdChZjsnX51g4XrHFOIXwfnCVPGlQk/k5giakcKsuxCObBRu6DSm9opw/O6slWbJdghQM4bBg== + +neo-async@^2.6.2: + version "2.6.2" + resolved "https://registry.npmmirror.com/neo-async/-/neo-async-2.6.2.tgz#b4aafb93e3aeb2d8174ca53cf163ab7d7308305f" + integrity sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw== + +next-tick@1, next-tick@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/next-tick/-/next-tick-1.1.0.tgz#1836ee30ad56d67ef281b22bd199f709449b35eb" + integrity sha512-CXdUiJembsNjuToQvxayPZF9Vqht7hewsvy2sOWafLvi2awflj9mOC6bHIg50orX8IJvWKY9wYQ/zB2kogPslQ== + +nice-try@^1.0.4: + version "1.0.5" + resolved "https://registry.npmmirror.com/nice-try/-/nice-try-1.0.5.tgz#a3378a7696ce7d223e88fc9b764bd7ef1089e366" + integrity sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ== + +nlcst-to-string@^2.0.0: + version "2.0.4" + resolved "https://registry.npmmirror.com/nlcst-to-string/-/nlcst-to-string-2.0.4.tgz#9315dfab80882bbfd86ddf1b706f53622dc400cc" + integrity sha512-3x3jwTd6UPG7vi5k4GEzvxJ5rDA7hVUIRNHPblKuMVP9Z3xmlsd9cgLcpAMkc5uPOBna82EeshROFhsPkbnTZg== + +no-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/no-case/-/no-case-3.0.4.tgz#d361fd5c9800f558551a8369fc0dcd4662b6124d" + integrity sha512-fgAN3jGAh+RoxUGZHTSOLJIqUc2wmoBwGR4tbpNAKmmovFoWq0OdRkb0VkldReO2a2iBT/OEulG9XSUc10r3zg== + dependencies: + lower-case "^2.0.2" + tslib "^2.0.3" + +node-abi@^3.3.0: + version "3.22.0" + resolved "https://registry.npmmirror.com/node-abi/-/node-abi-3.22.0.tgz#00b8250e86a0816576258227edbce7bbe0039362" + integrity sha512-u4uAs/4Zzmp/jjsD9cyFYDXeISfUWaAVWshPmDZOFOv4Xl4SbzTXm53I04C2uRueYJ+0t5PEtLH/owbn2Npf/w== + dependencies: + semver "^7.3.5" + +node-addon-api@^3.2.1: + version "3.2.1" + resolved "https://registry.npmmirror.com/node-addon-api/-/node-addon-api-3.2.1.tgz#81325e0a2117789c0128dab65e7e38f07ceba161" + integrity sha512-mmcei9JghVNDYydghQmeDX8KoAm0FAiYyIcUt/N4nhyAipB17pllZQDOJD2fotxABnt4Mdz+dKTO7eftLg4d0A== + +node-addon-api@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/node-addon-api/-/node-addon-api-4.3.0.tgz#52a1a0b475193e0928e98e0426a0d1254782b77f" + integrity sha512-73sE9+3UaLYYFmDsFZnqCInzPyh3MqIwZO9cw58yIqAZhONrrabrYyYe3TuIqtIiOuTXVhsGau8hcrhhwSsDIQ== + +node-addon-api@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/node-addon-api/-/node-addon-api-5.0.0.tgz#7d7e6f9ef89043befdb20c1989c905ebde18c501" + integrity sha512-CvkDw2OEnme7ybCykJpVcKH+uAOLV2qLqiyla128dN9TkEWfrYmxG6C2boDe5KcNQqZF3orkqzGgOMvZ/JNekA== + +node-fetch@2.6.1: + version "2.6.1" + resolved "https://registry.npmmirror.com/node-fetch/-/node-fetch-2.6.1.tgz#045bd323631f76ed2e2b55573394416b639a0052" + integrity sha512-V4aYg89jEoVRxRb2fJdAg8FHvI7cEyYdVAh94HH0UIK8oJxUfkjlDQN9RbMx+bEjP7+ggMiFRprSti032Oipxw== + +node-fetch@2.6.7, node-fetch@^2.6.1, node-fetch@^2.6.6, node-fetch@^2.6.7: + version "2.6.7" + resolved "https://registry.npmmirror.com/node-fetch/-/node-fetch-2.6.7.tgz#24de9fba827e3b4ae44dc8b20256a379160052ad" + integrity sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ== + dependencies: + whatwg-url "^5.0.0" + +node-gyp-build-optional-packages@5.0.2: + version "5.0.2" + resolved "https://registry.npmmirror.com/node-gyp-build-optional-packages/-/node-gyp-build-optional-packages-5.0.2.tgz#3de7d30bd1f9057b5dfbaeab4a4442b7fe9c5901" + integrity sha512-PiN4NWmlQPqvbEFcH/omQsswWQbe5Z9YK/zdB23irp5j2XibaA2IrGvpSWmVVG4qMZdmPdwPctSy4a86rOMn6g== + +node-gyp-build-optional-packages@5.0.3: + version "5.0.3" + resolved "https://registry.npmmirror.com/node-gyp-build-optional-packages/-/node-gyp-build-optional-packages-5.0.3.tgz#92a89d400352c44ad3975010368072b41ad66c17" + integrity sha512-k75jcVzk5wnnc/FMxsf4udAoTEUv2jY3ycfdSd3yWu6Cnd1oee6/CfZJApyscA4FJOmdoixWwiwOyf16RzD5JA== + +node-gyp-build-optional-packages@^4.3.2: + version "4.3.5" + resolved "https://registry.npmmirror.com/node-gyp-build-optional-packages/-/node-gyp-build-optional-packages-4.3.5.tgz#a1de0039f81ecacecefcbb4349cdb96842343b31" + integrity sha512-5ke7D8SiQsTQL7CkHpfR1tLwfqtKc0KYEmlnkwd40jHCASskZeS98qoZ1qDUns2aUQWikcjidRUs6PM/3iyN/w== + +node-gyp-build@^4.3.0: + version "4.4.0" + resolved "https://registry.npmmirror.com/node-gyp-build/-/node-gyp-build-4.4.0.tgz#42e99687ce87ddeaf3a10b99dc06abc11021f3f4" + integrity sha512-amJnQCcgtRVw9SvoebO3BKGESClrfXGCUTX9hSn1OuGQTQBOZmVd0Z0OlecpuRksKvbsUqALE8jls/ErClAPuQ== + +node-gyp@^7.1.0: + version "7.1.2" + resolved "https://registry.npmmirror.com/node-gyp/-/node-gyp-7.1.2.tgz#21a810aebb187120251c3bcec979af1587b188ae" + integrity sha512-CbpcIo7C3eMu3dL1c3d0xw449fHIGALIJsRP4DDPHpyiW8vcriNY7ubh9TE4zEKfSxscY7PjeFnshE7h75ynjQ== + dependencies: + env-paths "^2.2.0" + glob "^7.1.4" + graceful-fs "^4.2.3" + nopt "^5.0.0" + npmlog "^4.1.2" + request "^2.88.2" + rimraf "^3.0.2" + semver "^7.3.2" + tar "^6.0.2" + which "^2.0.2" + +node-int64@^0.4.0: + version "0.4.0" + resolved "https://registry.npmmirror.com/node-int64/-/node-int64-0.4.0.tgz#87a9065cdb355d3182d8f94ce11188b825c68a3b" + integrity sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw== + +node-object-hash@^2.3.10: + version "2.3.10" + resolved "https://registry.npmmirror.com/node-object-hash/-/node-object-hash-2.3.10.tgz#4b0c1a3a8239e955f0db71f8e00b38b5c0b33992" + integrity sha512-jY5dPJzw6NHd/KPSfPKJ+IHoFS81/tJ43r34ZeNMXGzCOM8jwQDCD12HYayKIB6MuznrnqIYy2e891NA2g0ibA== + +node-releases@^2.0.5: + version "2.0.5" + resolved "https://registry.npmmirror.com/node-releases/-/node-releases-2.0.5.tgz#280ed5bc3eba0d96ce44897d8aee478bfb3d9666" + integrity sha512-U9h1NLROZTq9uE1SNffn6WuPDg8icmi3ns4rEl/oTfIle4iLjTliCzgTsbaIFMq/Xn078/lfY/BL0GWZ+psK4Q== + +node-sass@^6.0.1: + version "6.0.1" + resolved "https://registry.npmmirror.com/node-sass/-/node-sass-6.0.1.tgz#cad1ccd0ce63e35c7181f545d8b986f3a9a887fe" + integrity sha512-f+Rbqt92Ful9gX0cGtdYwjTrWAaGURgaK5rZCWOgCNyGWusFYHhbqCCBoFBeat+HKETOU02AyTxNhJV0YZf2jQ== + dependencies: + async-foreach "^0.1.3" + chalk "^1.1.1" + cross-spawn "^7.0.3" + gaze "^1.0.0" + get-stdin "^4.0.1" + glob "^7.0.3" + lodash "^4.17.15" + meow "^9.0.0" + nan "^2.13.2" + node-gyp "^7.1.0" + npmlog "^4.0.0" + request "^2.88.0" + sass-graph "2.2.5" + stdout-stream "^1.4.0" + "true-case-path" "^1.0.2" + +nopt@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/nopt/-/nopt-5.0.0.tgz#530942bb58a512fccafe53fe210f13a25355dc88" + integrity sha512-Tbj67rffqceeLpcRXrT7vKAN8CwfPeIBgM7E6iBkmKLV7bEMwpGgYLGv0jACUsECaa/vuxP0IjEont6umdMgtQ== + dependencies: + abbrev "1" + +normalize-package-data@^2.5.0: + version "2.5.0" + resolved "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-2.5.0.tgz#e66db1838b200c1dfc233225d12cb36520e234a8" + integrity sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA== + dependencies: + hosted-git-info "^2.1.4" + resolve "^1.10.0" + semver "2 || 3 || 4 || 5" + validate-npm-package-license "^3.0.1" + +normalize-package-data@^3.0.0: + version "3.0.3" + resolved "https://registry.npmmirror.com/normalize-package-data/-/normalize-package-data-3.0.3.tgz#dbcc3e2da59509a0983422884cd172eefdfa525e" + integrity sha512-p2W1sgqij3zMMyRC067Dg16bfzVH+w7hyegmpIvZ4JNjqtGOVAIvLmjBx3yP7YTe9vKJgkoNOPjwQGogDoMXFA== + dependencies: + hosted-git-info "^4.0.1" + is-core-module "^2.5.0" + semver "^7.3.4" + validate-npm-package-license "^3.0.1" + +normalize-path@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/normalize-path/-/normalize-path-2.1.1.tgz#1ab28b556e198363a8c1a6f7e6fa20137fe6aed9" + integrity sha512-3pKJwH184Xo/lnH6oyP1q2pMd7HcypqqmRs91/6/i2CGtWwIKGCkOOMTm/zXbgTEWHw1uNpNi/igc3ePOYHb6w== + dependencies: + remove-trailing-separator "^1.0.1" + +normalize-path@^3.0.0, normalize-path@~3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" + integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== + +normalize-range@^0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/normalize-range/-/normalize-range-0.1.2.tgz#2d10c06bdfd312ea9777695a4d28439456b75942" + integrity sha512-bdok/XvKII3nUpklnV6P2hxtMNrCboOjAcyBuQnWEhO665FwrSNRxU+AqpsyvO6LgGYPspN+lu5CLtw4jPRKNA== + +normalize-url@^4.1.0: + version "4.5.1" + resolved "https://registry.npmmirror.com/normalize-url/-/normalize-url-4.5.1.tgz#0dd90cf1288ee1d1313b87081c9a5932ee48518a" + integrity sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA== + +normalize-url@^6.0.1, normalize-url@^6.1.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/normalize-url/-/normalize-url-6.1.0.tgz#40d0885b535deffe3f3147bec877d05fe4c5668a" + integrity sha512-DlL+XwOy3NxAQ8xuC0okPgK46iuVNAK01YN7RueYBqqFeGsBjV9XmCAzAdgt+667bCl5kPh9EqKKDwnaPG1I7A== + +not@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/not/-/not-0.1.0.tgz#c9691c1746c55dcfbe54cbd8bd4ff041bc2b519d" + integrity sha512-5PDmaAsVfnWUgTUbJ3ERwn7u79Z0dYxN9ErxCpVJJqe2RK0PJ3z+iFUxuqjwtlDDegXvtWoxD/3Fzxox7tFGWA== + +npm-run-path@^2.0.0: + version "2.0.2" + resolved "https://registry.npmmirror.com/npm-run-path/-/npm-run-path-2.0.2.tgz#35a9232dfa35d7067b4cb2ddf2357b1871536c5f" + integrity sha512-lJxZYlT4DW/bRUtFh1MQIWqmLwQfAxnqWG4HhEdjMlkrJYnJn0Jrr2u3mgxqaWsdiBc76TYkTG/mhrnYTuzfHw== + dependencies: + path-key "^2.0.0" + +npm-run-path@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/npm-run-path/-/npm-run-path-4.0.1.tgz#b7ecd1e5ed53da8e37a55e1c2269e0b97ed748ea" + integrity sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw== + dependencies: + path-key "^3.0.0" + +npmlog@^4.0.0, npmlog@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/npmlog/-/npmlog-4.1.2.tgz#08a7f2a8bf734604779a9efa4ad5cc717abb954b" + integrity sha512-2uUqazuKlTaSI/dC8AzicUck7+IrEaOnN/e0jd3Xtt1KcGpwx30v50mL7oPyr/h9bL3E4aZccVwpwP+5W9Vjkg== + dependencies: + are-we-there-yet "~1.1.2" + console-control-strings "~1.1.0" + gauge "~2.7.3" + set-blocking "~2.0.0" + +nth-check@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/nth-check/-/nth-check-1.0.2.tgz#b2bd295c37e3dd58a3bf0700376663ba4d9cf05c" + integrity sha512-WeBOdju8SnzPN5vTUJYxYUxLeXpCaVP5i5e0LF8fg7WORF2Wd7wFX/pk0tYZk7s8T+J7VLy0Da6J1+wCT0AtHg== + dependencies: + boolbase "~1.0.0" + +nth-check@^2.0.0, nth-check@^2.0.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/nth-check/-/nth-check-2.1.1.tgz#c9eab428effce36cd6b92c924bdb000ef1f1ed1d" + integrity sha512-lqjrjmaOoAnWfMmBPL+XNnynZh2+swxiX3WUE0s4yEHI6m+AwrK2UZOimIRl3X/4QctVqS8AiZjFqyOGrMXb/w== + dependencies: + boolbase "^1.0.0" + +null-loader@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/null-loader/-/null-loader-4.0.1.tgz#8e63bd3a2dd3c64236a4679428632edd0a6dbc6a" + integrity sha512-pxqVbi4U6N26lq+LmgIbB5XATP0VdZKOG25DhHi8btMmJJefGArFyDg1yc4U3hWCJbMqSrw0qyrz1UQX+qYXqg== + dependencies: + loader-utils "^2.0.0" + schema-utils "^3.0.0" + +nullthrows@^1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/nullthrows/-/nullthrows-1.1.1.tgz#7818258843856ae971eae4208ad7d7eb19a431b1" + integrity sha512-2vPPEi+Z7WqML2jZYddDIfy5Dqb0r2fze2zTxNNknZaFpVHU3mFB3R+DWeJWGVx0ecvttSGlJTI+WG+8Z4cDWw== + +number-is-nan@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/number-is-nan/-/number-is-nan-1.0.1.tgz#097b602b53422a522c1afb8790318336941a011d" + integrity sha512-4jbtZXNAsfZbAHiiqjLPBiCl16dES1zI4Hpzzxw61Tk+loF+sBDBKx1ICKKKwIqQ7M0mFn1TmkN7euSncWgHiQ== + +oauth-sign@~0.9.0: + version "0.9.0" + resolved "https://registry.npmmirror.com/oauth-sign/-/oauth-sign-0.9.0.tgz#47a7b016baa68b5fa0ecf3dee08a85c679ac6455" + integrity sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ== + +object-assign@^4, object-assign@^4.1.0, object-assign@^4.1.1: + version "4.1.1" + resolved "https://registry.npmmirror.com/object-assign/-/object-assign-4.1.1.tgz#2109adc7965887cfc05cbbd442cac8bfbb360863" + integrity sha512-rJgTQnkUnH1sFw8yT6VSU3zD3sWmu6sZhIseY8VX+GRu3P6F7Fu+JNDoXfklElbLJSnc3FUQHVe4cU5hj+BcUg== + +object-hash@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/object-hash/-/object-hash-3.0.0.tgz#73f97f753e7baffc0e2cc9d6e079079744ac82e9" + integrity sha512-RSn9F68PjH9HqtltsSnqYC1XXoWe9Bju5+213R98cNGttag9q9yAOTzdbsqvIa7aNm5WffBZFpWYr2aWrklWAw== + +object-inspect@^1.12.0, object-inspect@^1.9.0: + version "1.12.2" + resolved "https://registry.npmmirror.com/object-inspect/-/object-inspect-1.12.2.tgz#c0641f26394532f28ab8d796ab954e43c009a8ea" + integrity sha512-z+cPxW0QGUp0mcqcsgQyLVRDoXFQbXOwBaqyF7VIgI4TWNQsDHrBpUQslRmIfAoYWdYzs6UlKJtB2XJpTaNSpQ== + +object-keys@^1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.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.npmmirror.com/object-keys/-/object-keys-0.4.0.tgz#28a6aae7428dd2c3a92f3d95f21335dd204e0336" + integrity sha512-ncrLw+X55z7bkl5PnUvHwFK9FcGuFYo9gtjws2XtSzL+aZ8tm830P60WJ0dSmFVaSalWieW5MD7kEdnXda9yJw== + +object.assign@^4.1.0, object.assign@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/object.assign/-/object.assign-4.1.2.tgz#0ed54a342eceb37b38ff76eb831a0e788cb63940" + integrity sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ== + dependencies: + call-bind "^1.0.0" + define-properties "^1.1.3" + has-symbols "^1.0.1" + object-keys "^1.1.1" + +object.entries@^1.1.5: + version "1.1.5" + resolved "https://registry.npmmirror.com/object.entries/-/object.entries-1.1.5.tgz#e1acdd17c4de2cd96d5a08487cfb9db84d881861" + integrity sha512-TyxmjUoZggd4OrrU1W66FMDG6CuqJxsFvymeyXI51+vQLN67zYfZseptRge703kKQdo4uccgAKebXFcRCzk4+g== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.1" + +object.fromentries@^2.0.5: + version "2.0.5" + resolved "https://registry.npmmirror.com/object.fromentries/-/object.fromentries-2.0.5.tgz#7b37b205109c21e741e605727fe8b0ad5fa08251" + integrity sha512-CAyG5mWQRRiBU57Re4FKoTBjXfDoNwdFVH2Y1tS9PqCsfUTymAohOkEMSG3aRNKmv4lV3O7p1et7c187q6bynw== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.1" + +object.getownpropertydescriptors@^2.1.0: + version "2.1.4" + resolved "https://registry.npmmirror.com/object.getownpropertydescriptors/-/object.getownpropertydescriptors-2.1.4.tgz#7965e6437a57278b587383831a9b829455a4bc37" + integrity sha512-sccv3L/pMModT6dJAYF3fzGMVcb38ysQ0tEE6ixv2yXJDtEIPph268OlAdJj5/qZMZDq2g/jqvwppt36uS/uQQ== + dependencies: + array.prototype.reduce "^1.0.4" + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.1" + +object.hasown@^1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/object.hasown/-/object.hasown-1.1.1.tgz#ad1eecc60d03f49460600430d97f23882cf592a3" + integrity sha512-LYLe4tivNQzq4JdaWW6WO3HMZZJWzkkH8fnI6EebWl0VZth2wL2Lovm74ep2/gZzlaTdV62JZHEqHQ2yVn8Q/A== + dependencies: + define-properties "^1.1.4" + es-abstract "^1.19.5" + +object.values@^1.1.0, object.values@^1.1.5: + version "1.1.5" + resolved "https://registry.npmmirror.com/object.values/-/object.values-1.1.5.tgz#959f63e3ce9ef108720333082131e4a459b716ac" + integrity sha512-QUZRW0ilQ3PnPpbNtgdNV1PDbEqLIiSFB3l+EnGtBQ/8SUTLj1PZwtQHABZtLgwpJZTSZhuGLOGk57Drx2IvYg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.1" + +objectFitPolyfill@^2.3.5: + version "2.3.5" + resolved "https://registry.npmmirror.com/objectFitPolyfill/-/objectFitPolyfill-2.3.5.tgz#be8c83064aabfa4e88780f776c2013c48ce1f745" + integrity sha512-8Quz071ZmGi0QWEG4xB3Bv5Lpw6K0Uca87FLoLMKMWjB6qIq9IyBegP3b/VLNxv2WYvIMGoeUQ+c6ibUkNa8TA== + +omggif@^1.0.10, omggif@^1.0.9: + version "1.0.10" + resolved "https://registry.npmmirror.com/omggif/-/omggif-1.0.10.tgz#ddaaf90d4a42f532e9e7cb3a95ecdd47f17c7b19" + integrity sha512-LMJTtvgc/nugXj0Vcrrs68Mn2D1r0zf630VNtqtpI1FEO7e+O9FP4gqs9AcnBaSEeoHIPm28u6qgPR0oyEpGSw== + +on-finished@2.4.1, on-finished@^2.3.0: + version "2.4.1" + resolved "https://registry.npmmirror.com/on-finished/-/on-finished-2.4.1.tgz#58c8c44116e54845ad57f14ab10b03533184ac3f" + integrity sha512-oVlzkg3ENAhCk2zdv7IJwd/QUD4z2RxRwpkcGY8psCVcCYZNq4wYnVWALHM+brtuJjePWiYF/ClmuDr8Ch5+kg== + dependencies: + ee-first "1.1.1" + +on-headers@~1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/on-headers/-/on-headers-1.0.2.tgz#772b0ae6aaa525c399e489adfad90c403eb3c28f" + integrity sha512-pZAE+FJLoyITytdqK0U5s+FIpjN0JP3OzFi/u8Rx+EV5/W+JTWGXG8xFzevE7AjBfDqHv/8vL8qQsIhHnqRkrA== + +once@^1.3.0, once@^1.3.1, once@^1.4.0: + version "1.4.0" + resolved "https://registry.npmmirror.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1" + integrity sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w== + dependencies: + wrappy "1" + +onetime@^5.1.0, onetime@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" + integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== + dependencies: + mimic-fn "^2.1.0" + +open@^7.0.3: + version "7.4.2" + resolved "https://registry.npmmirror.com/open/-/open-7.4.2.tgz#b8147e26dcf3e426316c730089fd71edd29c2321" + integrity sha512-MVHddDVweXZF3awtlAS+6pgKLlm/JgxZ90+/NBurBoQctVOOB/zDdVjcyPzQ+0laDGbsWgrRkflI65sQeOgT9Q== + dependencies: + is-docker "^2.0.0" + is-wsl "^2.1.1" + +open@^8.4.0: + version "8.4.0" + resolved "https://registry.npmmirror.com/open/-/open-8.4.0.tgz#345321ae18f8138f82565a910fdc6b39e8c244f8" + integrity sha512-XgFPPM+B28FtCCgSb9I+s9szOC1vZRSwgWsRUA5ylIxRTgKozqjOCrVOqGsYABPYK5qnfqClxZTFBa8PKt2v6Q== + dependencies: + define-lazy-prop "^2.0.0" + is-docker "^2.1.1" + is-wsl "^2.2.0" + +opentracing@^0.14.5: + version "0.14.7" + resolved "https://registry.npmmirror.com/opentracing/-/opentracing-0.14.7.tgz#25d472bd0296dc0b64d7b94cbc995219031428f5" + integrity sha512-vz9iS7MJ5+Bp1URw8Khvdyw1H/hGvzHWlKQ7eRrQojSCDL1/SrWfrY9QebLw97n2deyRtzHRC3MkQfVNUCo91Q== + +optionator@^0.9.1: + version "0.9.1" + resolved "https://registry.npmmirror.com/optionator/-/optionator-0.9.1.tgz#4f236a6373dae0566a6d43e1326674f50c291499" + integrity sha512-74RlY5FCnhq4jRxVUPKDaRwrVNXMqsGsiW6AJw4XK8hmtm10wC0ypZBLw5IIp85NZMr91+qd1RvvENwg7jjRFw== + dependencies: + deep-is "^0.1.3" + fast-levenshtein "^2.0.6" + levn "^0.4.1" + prelude-ls "^1.2.1" + type-check "^0.4.0" + word-wrap "^1.2.3" + +ordered-binary@^1.2.4: + version "1.2.5" + resolved "https://registry.npmmirror.com/ordered-binary/-/ordered-binary-1.2.5.tgz#6208c45067eae9d14b8f44791a1d7037adad9147" + integrity sha512-djRmZoEpOGvIRW7ufsCDHtvcUa18UC9TxnPbHhSVFZHsoyg0dtut1bWtBZ/fmxdPN62oWXrV6adM7NoWU+CneA== + +os-tmpdir@~1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/os-tmpdir/-/os-tmpdir-1.0.2.tgz#bbe67406c79aa85c5cfec766fe5734555dfa1274" + integrity sha512-D2FR03Vir7FIu45XBY20mTb+/ZSWB00sjU9jdQXt83gDrI4Ztz5Fs7/yy74g2N5SVQY4xY1qDr4rNddwYRVX0g== + +p-cancelable@^1.0.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/p-cancelable/-/p-cancelable-1.1.0.tgz#d078d15a3af409220c886f1d9a0ca2e441ab26cc" + integrity sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw== + +p-cancelable@^2.0.0: + version "2.1.1" + resolved "https://registry.npmmirror.com/p-cancelable/-/p-cancelable-2.1.1.tgz#aab7fbd416582fa32a3db49859c122487c5ed2cf" + integrity sha512-BZOr3nRQHOntUjTrH8+Lh54smKHoHyur8We1V8DSMVrl5A2malOOwuJRnKRDjSnkoeBh4at6BwEnb5I7Jl31wg== + +p-defer@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/p-defer/-/p-defer-1.0.0.tgz#9f6eb182f6c9aa8cd743004a7d4f96b196b0fb0c" + integrity sha512-wB3wfAxZpk2AzOfUMJNL+d36xothRSyj8EXOa4f6GMqYDN9BJaaSISbsk+wS9abmnebVw95C2Kb5t85UmpCxuw== + +p-defer@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/p-defer/-/p-defer-3.0.0.tgz#d1dceb4ee9b2b604b1d94ffec83760175d4e6f83" + integrity sha512-ugZxsxmtTln604yeYd29EGrNhazN2lywetzpKhfmQjW/VJmhpDmWbiX+h0zL8V91R0UXkhb3KtPmyq9PZw3aYw== + +p-finally@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/p-finally/-/p-finally-1.0.0.tgz#3fbcfb15b899a44123b34b6dcc18b724336a2cae" + integrity sha512-LICb2p9CB7FS+0eR1oqWnHhp0FljGLZCWBE9aix0Uye9W8LTQPwMTYVGWQWIw9RdQiDg4+epXQODwIYJtSJaow== + +p-limit@3.1.0, p-limit@^3.0.2: + version "3.1.0" + resolved "https://registry.npmmirror.com/p-limit/-/p-limit-3.1.0.tgz#e1daccbe78d0d1388ca18c64fea38e3e57e3706b" + integrity sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ== + dependencies: + yocto-queue "^0.1.0" + +p-limit@^1.1.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/p-limit/-/p-limit-1.3.0.tgz#b86bd5f0c25690911c7590fcbfc2010d54b3ccb8" + integrity sha512-vvcXsLAJ9Dr5rQOPk7toZQZJApBl2K4J6dANSsEuh6QI41JYcsS/qhTGa9ErIUUgK3WNQoJYvylxvjqmiqEA9Q== + dependencies: + p-try "^1.0.0" + +p-limit@^2.0.0, p-limit@^2.2.0: + version "2.3.0" + resolved "https://registry.npmmirror.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" + integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== + dependencies: + p-try "^2.0.0" + +p-locate@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/p-locate/-/p-locate-2.0.0.tgz#20a0103b222a70c8fd39cc2e580680f3dde5ec43" + integrity sha512-nQja7m7gSKuewoVRen45CtVfODR3crN3goVQ0DDZ9N3yHxgpkuBhZqsaiotSQRrADUrne346peY7kT3TSACykg== + dependencies: + p-limit "^1.1.0" + +p-locate@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/p-locate/-/p-locate-3.0.0.tgz#322d69a05c0264b25997d9f40cd8a891ab0064a4" + integrity sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ== + dependencies: + p-limit "^2.0.0" + +p-locate@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" + integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== + dependencies: + p-limit "^2.2.0" + +p-locate@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/p-locate/-/p-locate-5.0.0.tgz#83c8315c6785005e3bd021839411c9e110e6d834" + integrity sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw== + dependencies: + p-limit "^3.0.2" + +p-try@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/p-try/-/p-try-1.0.0.tgz#cbc79cdbaf8fd4228e13f621f2b1a237c1b207b3" + integrity sha512-U1etNYuMJoIz3ZXSrrySFjsXQTWOx2/jdi86L+2pRvph/qMKL6sbcCYdH23fqsbm8TH2Gn0OybpT4eSFlCVHww== + +p-try@^2.0.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" + integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== + +package-json@^6.3.0: + version "6.5.0" + resolved "https://registry.npmmirror.com/package-json/-/package-json-6.5.0.tgz#6feedaca35e75725876d0b0e64974697fed145b0" + integrity sha512-k3bdm2n25tkyxcjSKzB5x8kfVxlMdgsbPr0GkZcwHsLpba6cBjqCt1KlcChKEvxHIcTB1FVMuwoijZ26xex5MQ== + dependencies: + got "^9.6.0" + registry-auth-token "^4.0.0" + registry-url "^5.0.0" + semver "^6.2.0" + +pako@^1.0.5: + version "1.0.11" + resolved "https://registry.npmmirror.com/pako/-/pako-1.0.11.tgz#6c9599d340d54dfd3946380252a35705a6b992bf" + integrity sha512-4hLB8Py4zZce5s4yd9XzopqwVv/yGNhV1Bl8NTmCq1763HeK2+EwVTv+leGeL13Dnh2wfbqowVPXCIO0z4taYw== + +param-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/param-case/-/param-case-3.0.4.tgz#7d17fe4aa12bde34d4a77d91acfb6219caad01c5" + integrity sha512-RXlj7zCYokReqWpOPH9oYivUzLYZ5vAPIfEmCTNViosC78F8F0H9y7T7gG2M39ymgutxF5gcFEsyZQSph9Bp3A== + dependencies: + dot-case "^3.0.4" + tslib "^2.0.3" + +parent-module@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/parent-module/-/parent-module-1.0.1.tgz#691d2709e78c79fae3a156622452d00762caaaa2" + integrity sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g== + dependencies: + callsites "^3.0.0" + +parse-bmfont-ascii@^1.0.3: + version "1.0.6" + resolved "https://registry.npmmirror.com/parse-bmfont-ascii/-/parse-bmfont-ascii-1.0.6.tgz#11ac3c3ff58f7c2020ab22769079108d4dfa0285" + integrity sha512-U4RrVsUFCleIOBsIGYOMKjn9PavsGOXxbvYGtMOEfnId0SVNsgehXh1DxUdVPLoxd5mvcEtvmKs2Mmf0Mpa1ZA== + +parse-bmfont-binary@^1.0.5: + version "1.0.6" + resolved "https://registry.npmmirror.com/parse-bmfont-binary/-/parse-bmfont-binary-1.0.6.tgz#d038b476d3e9dd9db1e11a0b0e53a22792b69006" + integrity sha512-GxmsRea0wdGdYthjuUeWTMWPqm2+FAd4GI8vCvhgJsFnoGhTrLhXDDupwTo7rXVAgaLIGoVHDZS9p/5XbSqeWA== + +parse-bmfont-xml@^1.1.4: + version "1.1.4" + resolved "https://registry.npmmirror.com/parse-bmfont-xml/-/parse-bmfont-xml-1.1.4.tgz#015319797e3e12f9e739c4d513872cd2fa35f389" + integrity sha512-bjnliEOmGv3y1aMEfREMBJ9tfL3WR0i0CKPj61DnSLaoxWR3nLrsQrEbCId/8rF4NyRF0cCqisSVXyQYWM+mCQ== + dependencies: + xml-parse-from-string "^1.0.0" + xml2js "^0.4.5" + +parse-english@^4.0.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/parse-english/-/parse-english-4.2.0.tgz#037b68f34d1a1bdf3d33668b87791bdfc1f01e1e" + integrity sha512-jw5N6wZUZViIw3VLG/FUSeL3vDhfw5Q2g4E3nYC69Mm5ANbh9ZWd+eligQbeUoyObZM8neynTn3l14e09pjEWg== + dependencies: + nlcst-to-string "^2.0.0" + parse-latin "^4.0.0" + unist-util-modify-children "^2.0.0" + unist-util-visit-children "^1.0.0" + +parse-entities@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/parse-entities/-/parse-entities-2.0.0.tgz#53c6eb5b9314a1f4ec99fa0fdf7ce01ecda0cbe8" + integrity sha512-kkywGpCcRYhqQIchaWqZ875wzpS/bMKhz5HnN3p7wveJTkTtyAB/AlnS0f8DFSqYW1T82t6yEAkEcB+A1I3MbQ== + dependencies: + character-entities "^1.0.0" + character-entities-legacy "^1.0.0" + character-reference-invalid "^1.0.0" + is-alphanumerical "^1.0.0" + is-decimal "^1.0.0" + is-hexadecimal "^1.0.0" + +parse-filepath@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/parse-filepath/-/parse-filepath-1.0.2.tgz#a632127f53aaf3d15876f5872f3ffac763d6c891" + integrity sha512-FwdRXKCohSVeXqwtYonZTXtbGJKrn+HNyWDYVcp5yuJlesTwNH4rsmRZ+GrKAPJ5bLpRxESMeS+Rl0VCHRvB2Q== + dependencies: + is-absolute "^1.0.0" + map-cache "^0.2.0" + path-root "^0.1.1" + +parse-headers@^2.0.0: + version "2.0.5" + resolved "https://registry.npmmirror.com/parse-headers/-/parse-headers-2.0.5.tgz#069793f9356a54008571eb7f9761153e6c770da9" + integrity sha512-ft3iAoLOB/MlwbNXgzy43SWGP6sQki2jQvAyBg/zDFAgr9bfNWZIUj42Kw2eJIl8kEi4PbgE6U1Zau/HwI75HA== + +parse-json@^5.0.0: + version "5.2.0" + resolved "https://registry.npmmirror.com/parse-json/-/parse-json-5.2.0.tgz#c76fc66dee54231c962b22bcc8a72cf2f99753cd" + integrity sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg== + dependencies: + "@babel/code-frame" "^7.0.0" + error-ex "^1.3.1" + json-parse-even-better-errors "^2.3.0" + lines-and-columns "^1.1.6" + +parse-latin@^4.0.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/parse-latin/-/parse-latin-4.3.0.tgz#1a70fc5601743baa06c5f12253c334fc94b4a917" + integrity sha512-TYKL+K98dcAWoCw/Ac1yrPviU8Trk+/gmjQVaoWEFDZmVD4KRg6c/80xKqNNFQObo2mTONgF8trzAf2UTwKafw== + dependencies: + nlcst-to-string "^2.0.0" + unist-util-modify-children "^2.0.0" + unist-util-visit-children "^1.0.0" + +parse-path@^4.0.0: + version "4.0.4" + resolved "https://registry.npmmirror.com/parse-path/-/parse-path-4.0.4.tgz#4bf424e6b743fb080831f03b536af9fc43f0ffea" + integrity sha512-Z2lWUis7jlmXC1jeOG9giRO2+FsuyNipeQ43HAjqAZjwSe3SEf+q/84FGPHoso3kyntbxa4c4i77t3m6fGf8cw== + dependencies: + is-ssh "^1.3.0" + protocols "^1.4.0" + qs "^6.9.4" + query-string "^6.13.8" + +parse-srcset@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/parse-srcset/-/parse-srcset-1.0.2.tgz#f2bd221f6cc970a938d88556abc589caaaa2bde1" + integrity sha512-/2qh0lav6CmI15FzA3i/2Bzk2zCgQhGMkvhOhKNcBVQ1ldgpbfiNTVslmooUmWJcADi1f1kIeynbDRVzNlfR6Q== + +parse-url@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/parse-url/-/parse-url-6.0.0.tgz#f5dd262a7de9ec00914939220410b66cff09107d" + integrity sha512-cYyojeX7yIIwuJzledIHeLUBVJ6COVLeT4eF+2P6aKVzwvgKQPndCBv3+yQ7pcWjqToYwaligxzSYNNmGoMAvw== + dependencies: + is-ssh "^1.3.0" + normalize-url "^6.1.0" + parse-path "^4.0.0" + protocols "^1.4.0" + +parse5-htmlparser2-tree-adapter@^7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/parse5-htmlparser2-tree-adapter/-/parse5-htmlparser2-tree-adapter-7.0.0.tgz#23c2cc233bcf09bb7beba8b8a69d46b08c62c2f1" + integrity sha512-B77tOZrqqfUfnVcOrUvfdLbz4pu4RopLD/4vmu3HUPswwTA8OH0EMW9BlWR2B0RCoiZRAHEUu7IxeP1Pd1UU+g== + dependencies: + domhandler "^5.0.2" + parse5 "^7.0.0" + +parse5@^6.0.0: + version "6.0.1" + resolved "https://registry.npmmirror.com/parse5/-/parse5-6.0.1.tgz#e1a1c085c569b3dc08321184f19a39cc27f7c30b" + integrity sha512-Ofn/CTFzRGTTxwpNEs9PP93gXShHcTq255nzRYSKe8AkVpZY7e1fpmTfOyoIvjP5HG7Z2ZM7VS9PPhQGW2pOpw== + +parse5@^7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/parse5/-/parse5-7.0.0.tgz#51f74a5257f5fcc536389e8c2d0b3802e1bfa91a" + integrity sha512-y/t8IXSPWTuRZqXc0ajH/UwDj4mnqLEbSttNbThcFhGrZuOyoyvNBO85PBp2jQa55wY9d07PBNjsK8ZP3K5U6g== + dependencies: + entities "^4.3.0" + +parseqs@0.0.6: + version "0.0.6" + resolved "https://registry.npmmirror.com/parseqs/-/parseqs-0.0.6.tgz#8e4bb5a19d1cdc844a08ac974d34e273afa670d5" + integrity sha512-jeAGzMDbfSHHA091hr0r31eYfTig+29g3GKKE/PPbEQ65X0lmMwlEoqmhzu0iztID5uJpZsFlUPDP8ThPL7M8w== + +parseuri@0.0.6: + version "0.0.6" + resolved "https://registry.npmmirror.com/parseuri/-/parseuri-0.0.6.tgz#e1496e829e3ac2ff47f39a4dd044b32823c4a25a" + integrity sha512-AUjen8sAkGgao7UyCX6Ahv0gIK2fABKmYjvP4xmy5JaKvcbTRueIqIPHLAfq30xJddqSE033IOMUSOMCcK3Sow== + +parseurl@^1.3.3, parseurl@~1.3.3: + version "1.3.3" + resolved "https://registry.npmmirror.com/parseurl/-/parseurl-1.3.3.tgz#9da19e7bee8d12dff0513ed5b76957793bc2e8d4" + integrity sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ== + +pascal-case@^3.1.2: + version "3.1.2" + resolved "https://registry.npmmirror.com/pascal-case/-/pascal-case-3.1.2.tgz#b48e0ef2b98e205e7c1dae747d0b1508237660eb" + integrity sha512-uWlGT3YSnK9x3BQJaOdcZwrnV6hPpd8jFH1/ucpiLRPh/2zCVJKS19E4GvYHvaCcACn3foXZ0cLB9Wrx1KGe5g== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + +password-prompt@^1.0.4: + version "1.1.2" + resolved "https://registry.npmmirror.com/password-prompt/-/password-prompt-1.1.2.tgz#85b2f93896c5bd9e9f2d6ff0627fa5af3dc00923" + integrity sha512-bpuBhROdrhuN3E7G/koAju0WjVw9/uQOG5Co5mokNj0MiOSBVZS1JTwM4zl55hu0WFmIEFvO9cU9sJQiBIYeIA== + dependencies: + ansi-escapes "^3.1.0" + cross-spawn "^6.0.5" + +path-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/path-case/-/path-case-3.0.4.tgz#9168645334eb942658375c56f80b4c0cb5f82c6f" + integrity sha512-qO4qCFjXqVTrcbPt/hQfhTQ+VhFsqNKOPtytgNKkKxSoEp3XPUQ8ObFuePylOIok5gjn69ry8XiULxCwot3Wfg== + dependencies: + dot-case "^3.0.4" + tslib "^2.0.3" + +path-exists@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/path-exists/-/path-exists-3.0.0.tgz#ce0ebeaa5f78cb18925ea7d810d7b59b010fd515" + integrity sha512-bpC7GYwiDYQ4wYLe+FA8lhRjhQCMcQGuSgGGqDkg/QerRWw9CmGRT0iSOVRSZJ29NMLZgIzqaljJ63oaL4NIJQ== + +path-exists@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/path-exists/-/path-exists-4.0.0.tgz#513bdbe2d3b95d7762e8c1137efa195c6c61b5b3" + integrity sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w== + +path-is-absolute@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/path-is-absolute/-/path-is-absolute-1.0.1.tgz#174b9268735534ffbc7ace6bf53a5a9e1b5c5f5f" + integrity sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg== + +path-key@^2.0.0, path-key@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/path-key/-/path-key-2.0.1.tgz#411cadb574c5a140d3a4b1910d40d80cc9f40b40" + integrity sha512-fEHGKCSmUSDPv4uoj8AlD+joPlq3peND+HRYyxFz4KPw4z926S/b8rIuFs2FYJg3BwsxJf6A9/3eIdLaYC+9Dw== + +path-key@^3.0.0, path-key@^3.1.0: + version "3.1.1" + resolved "https://registry.npmmirror.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" + integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== + +path-parse@^1.0.6, path-parse@^1.0.7: + version "1.0.7" + resolved "https://registry.npmmirror.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" + integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== + +path-root-regex@^0.1.0: + version "0.1.2" + resolved "https://registry.npmmirror.com/path-root-regex/-/path-root-regex-0.1.2.tgz#bfccdc8df5b12dc52c8b43ec38d18d72c04ba96d" + integrity sha512-4GlJ6rZDhQZFE0DPVKh0e9jmZ5egZfxTkp7bcRDuPlJXbAwhxcl2dINPUAsjLdejqaLsCeg8axcLjIbvBjN4pQ== + +path-root@^0.1.1: + version "0.1.1" + resolved "https://registry.npmmirror.com/path-root/-/path-root-0.1.1.tgz#9a4a6814cac1c0cd73360a95f32083c8ea4745b7" + integrity sha512-QLcPegTHF11axjfojBIoDygmS2E3Lf+8+jI6wOVmNVenrKSo3mFdSGiIgdSHenczw3wPtlVMQaFVwGmM7BJdtg== + dependencies: + path-root-regex "^0.1.0" + +path-to-regexp@0.1.7: + version "0.1.7" + resolved "https://registry.npmmirror.com/path-to-regexp/-/path-to-regexp-0.1.7.tgz#df604178005f522f15eb4490e7247a1bfaa67f8c" + integrity sha512-5DFkuoqlv1uYQKxy8omFBeJPQcdoE07Kv2sferDCrAq1ohOU+MSDswDIbnx3YAM60qIOnYa53wBhXW0EbMonrQ== + +path-type@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b" + integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw== + +peek-readable@^4.1.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/peek-readable/-/peek-readable-4.1.0.tgz#4ece1111bf5c2ad8867c314c81356847e8a62e72" + integrity sha512-ZI3LnwUv5nOGbQzD9c2iDG6toheuXSZP5esSHBjopsXH4dg19soufvpUGA3uohi5anFtGb2lhAVdHzH6R/Evvg== + +performance-now@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/performance-now/-/performance-now-2.1.0.tgz#6309f4e0e5fa913ec1c69307ae364b4b377c9e7b" + integrity sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow== + +phin@^2.9.1: + version "2.9.3" + resolved "https://registry.npmmirror.com/phin/-/phin-2.9.3.tgz#f9b6ac10a035636fb65dfc576aaaa17b8743125c" + integrity sha512-CzFr90qM24ju5f88quFC/6qohjC144rehe5n6DH900lgXmUe86+xCKc10ev56gRKC4/BkHUoG4uSiQgBiIXwDA== + +physical-cpu-count@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/physical-cpu-count/-/physical-cpu-count-2.0.0.tgz#18de2f97e4bf7a9551ad7511942b5496f7aba660" + integrity sha512-rxJOljMuWtYlvREBmd6TZYanfcPhNUKtGDZBjBBS8WG1dpN2iwPsRJZgQqN/OtJuiQckdRFOfzogqJClTrsi7g== + +picocolors@^0.2.1: + version "0.2.1" + resolved "https://registry.npmmirror.com/picocolors/-/picocolors-0.2.1.tgz#570670f793646851d1ba135996962abad587859f" + integrity sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA== + +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + +picomatch@^2.0.4, picomatch@^2.2.1, picomatch@^2.3.1: + version "2.3.1" + resolved "https://registry.npmmirror.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" + integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== + +pify@^2.3.0: + version "2.3.0" + resolved "https://registry.npmmirror.com/pify/-/pify-2.3.0.tgz#ed141a6ac043a849ea588498e7dca8b15330e90c" + integrity sha512-udgsAY+fTnvv7kI7aaxbqwWNb0AHiB0qBO89PZKPkoTmGOgdbrHDKD+0B2X4uTfJ/FT1R09r9gTsjUjNJotuog== + +pixelmatch@^4.0.2: + version "4.0.2" + resolved "https://registry.npmmirror.com/pixelmatch/-/pixelmatch-4.0.2.tgz#8f47dcec5011b477b67db03c243bc1f3085e8854" + integrity sha512-J8B6xqiO37sU/gkcMglv6h5Jbd9xNER7aHzpfRdNmV4IbQBzBpe4l9XmbG+xPF/znacgu2jfEw+wHffaq/YkXA== + dependencies: + pngjs "^3.0.0" + +pkg-dir@^4.1.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" + integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== + dependencies: + find-up "^4.0.0" + +pkg-up@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/pkg-up/-/pkg-up-3.1.0.tgz#100ec235cc150e4fd42519412596a28512a0def5" + integrity sha512-nDywThFk1i4BQK4twPQ6TA4RT8bDY96yeuCVBWL3ePARCiEKDRSrNGbFIgUJpLp+XeIR65v8ra7WuJOFUBtkMA== + dependencies: + find-up "^3.0.0" + +platform@^1.3.6: + version "1.3.6" + resolved "https://registry.npmmirror.com/platform/-/platform-1.3.6.tgz#48b4ce983164b209c2d45a107adb31f473a6e7a7" + integrity sha512-fnWVljUchTro6RiCFvCXBbNhJc2NijN7oIQxbwsyL0buWJPG85v81ehlHI9fXrJsMNgTofEoWIQeClKpgxFLrg== + +pngjs@^3.0.0, pngjs@^3.3.3: + version "3.4.0" + resolved "https://registry.npmmirror.com/pngjs/-/pngjs-3.4.0.tgz#99ca7d725965fb655814eaf65f38f12bbdbf555f" + integrity sha512-NCrCHhWmnQklfH4MtJMRjZ2a8c80qXeMlQMv2uVp9ISJMTt562SbGd6n2oq0PaPgKm7Z6pL9E2UlLIhC+SHL3w== + +postcss-calc@^8.2.3: + version "8.2.4" + resolved "https://registry.npmmirror.com/postcss-calc/-/postcss-calc-8.2.4.tgz#77b9c29bfcbe8a07ff6693dc87050828889739a5" + integrity sha512-SmWMSJmB8MRnnULldx0lQIyhSNvuDl9HfrZkaqqE/WHAhToYsAvDq+yAsA/kIyINDszOp3Rh0GFoNuH5Ypsm3Q== + dependencies: + postcss-selector-parser "^6.0.9" + postcss-value-parser "^4.2.0" + +postcss-colormin@^5.3.0: + version "5.3.0" + resolved "https://registry.npmmirror.com/postcss-colormin/-/postcss-colormin-5.3.0.tgz#3cee9e5ca62b2c27e84fce63affc0cfb5901956a" + integrity sha512-WdDO4gOFG2Z8n4P8TWBpshnL3JpmNmJwdnfP2gbk2qBA8PWwOYcmjmI/t3CmMeL72a7Hkd+x/Mg9O2/0rD54Pg== + dependencies: + browserslist "^4.16.6" + caniuse-api "^3.0.0" + colord "^2.9.1" + postcss-value-parser "^4.2.0" + +postcss-convert-values@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/postcss-convert-values/-/postcss-convert-values-5.1.2.tgz#31586df4e184c2e8890e8b34a0b9355313f503ab" + integrity sha512-c6Hzc4GAv95B7suy4udszX9Zy4ETyMCgFPUDtWjdFTKH1SE9eFY/jEpHSwTH1QPuwxHpWslhckUQWbNRM4ho5g== + dependencies: + browserslist "^4.20.3" + postcss-value-parser "^4.2.0" + +postcss-discard-comments@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/postcss-discard-comments/-/postcss-discard-comments-5.1.2.tgz#8df5e81d2925af2780075840c1526f0660e53696" + integrity sha512-+L8208OVbHVF2UQf1iDmRcbdjJkuBF6IS29yBDSiWUIzpYaAhtNl6JYnYm12FnkeCwQqF5LeklOu6rAqgfBZqQ== + +postcss-discard-duplicates@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-discard-duplicates/-/postcss-discard-duplicates-5.1.0.tgz#9eb4fe8456706a4eebd6d3b7b777d07bad03e848" + integrity sha512-zmX3IoSI2aoenxHV6C7plngHWWhUOV3sP1T8y2ifzxzbtnuhk1EdPwm0S1bIUNaJ2eNbWeGLEwzw8huPD67aQw== + +postcss-discard-empty@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/postcss-discard-empty/-/postcss-discard-empty-5.1.1.tgz#e57762343ff7f503fe53fca553d18d7f0c369c6c" + integrity sha512-zPz4WljiSuLWsI0ir4Mcnr4qQQ5e1Ukc3i7UfE2XcrwKK2LIPIqE5jxMRxO6GbI3cv//ztXDsXwEWT3BHOGh3A== + +postcss-discard-overridden@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-discard-overridden/-/postcss-discard-overridden-5.1.0.tgz#7e8c5b53325747e9d90131bb88635282fb4a276e" + integrity sha512-21nOL7RqWR1kasIVdKs8HNqQJhFxLsyRfAnUDm4Fe4t4mCWL9OJiHvlHPjcd8zc5Myu89b/7wZDnOSjFgeWRtw== + +postcss-flexbugs-fixes@^5.0.2: + version "5.0.2" + resolved "https://registry.npmmirror.com/postcss-flexbugs-fixes/-/postcss-flexbugs-fixes-5.0.2.tgz#2028e145313074fc9abe276cb7ca14e5401eb49d" + integrity sha512-18f9voByak7bTktR2QgDveglpn9DTbBWPUzSOe9g0N4WR/2eSt6Vrcbf0hmspvMI6YWGywz6B9f7jzpFNJJgnQ== + +postcss-import@^14.1.0: + version "14.1.0" + resolved "https://registry.npmmirror.com/postcss-import/-/postcss-import-14.1.0.tgz#a7333ffe32f0b8795303ee9e40215dac922781f0" + integrity sha512-flwI+Vgm4SElObFVPpTIT7SU7R3qk2L7PyduMcokiaVKuWv9d/U+Gm/QAd8NDLuykTWTkcrjOeD2Pp1rMeBTGw== + dependencies: + postcss-value-parser "^4.0.0" + read-cache "^1.0.0" + resolve "^1.1.7" + +postcss-js@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/postcss-js/-/postcss-js-4.0.0.tgz#31db79889531b80dc7bc9b0ad283e418dce0ac00" + integrity sha512-77QESFBwgX4irogGVPgQ5s07vLvFqWr228qZY+w6lW599cRlK/HmnlivnnVUxkjHnCu4J16PDMHcH+e+2HbvTQ== + dependencies: + camelcase-css "^2.0.1" + +postcss-load-config@^3.1.4: + version "3.1.4" + resolved "https://registry.npmmirror.com/postcss-load-config/-/postcss-load-config-3.1.4.tgz#1ab2571faf84bb078877e1d07905eabe9ebda855" + integrity sha512-6DiM4E7v4coTE4uzA8U//WhtPwyhiim3eyjEMFCnUpzbrkK9wJHgKDT2mR+HbtSrd/NubVaYTOpSpjUl8NQeRg== + dependencies: + lilconfig "^2.0.5" + yaml "^1.10.2" + +postcss-loader@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/postcss-loader/-/postcss-loader-4.3.0.tgz#2c4de9657cd4f07af5ab42bd60a673004da1b8cc" + integrity sha512-M/dSoIiNDOo8Rk0mUqoj4kpGq91gcxCfb9PoyZVdZ76/AuhxylHDYZblNE8o+EQ9AMSASeMFEKxZf5aU6wlx1Q== + dependencies: + cosmiconfig "^7.0.0" + klona "^2.0.4" + loader-utils "^2.0.0" + schema-utils "^3.0.0" + semver "^7.3.4" + +postcss-loader@^5.3.0: + version "5.3.0" + resolved "https://registry.npmmirror.com/postcss-loader/-/postcss-loader-5.3.0.tgz#1657f869e48d4fdb018a40771c235e499ee26244" + integrity sha512-/+Z1RAmssdiSLgIZwnJHwBMnlABPgF7giYzTN2NOfr9D21IJZ4mQC1R2miwp80zno9M4zMD/umGI8cR+2EL5zw== + dependencies: + cosmiconfig "^7.0.0" + klona "^2.0.4" + semver "^7.3.4" + +postcss-merge-longhand@^5.1.5: + version "5.1.5" + resolved "https://registry.npmmirror.com/postcss-merge-longhand/-/postcss-merge-longhand-5.1.5.tgz#b0e03bee3b964336f5f33c4fc8eacae608e91c05" + integrity sha512-NOG1grw9wIO+60arKa2YYsrbgvP6tp+jqc7+ZD5/MalIw234ooH2C6KlR6FEn4yle7GqZoBxSK1mLBE9KPur6w== + dependencies: + postcss-value-parser "^4.2.0" + stylehacks "^5.1.0" + +postcss-merge-rules@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/postcss-merge-rules/-/postcss-merge-rules-5.1.2.tgz#7049a14d4211045412116d79b751def4484473a5" + integrity sha512-zKMUlnw+zYCWoPN6yhPjtcEdlJaMUZ0WyVcxTAmw3lkkN/NDMRkOkiuctQEoWAOvH7twaxUUdvBWl0d4+hifRQ== + dependencies: + browserslist "^4.16.6" + caniuse-api "^3.0.0" + cssnano-utils "^3.1.0" + postcss-selector-parser "^6.0.5" + +postcss-minify-font-values@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-minify-font-values/-/postcss-minify-font-values-5.1.0.tgz#f1df0014a726083d260d3bd85d7385fb89d1f01b" + integrity sha512-el3mYTgx13ZAPPirSVsHqFzl+BBBDrXvbySvPGFnQcTI4iNslrPaFq4muTkLZmKlGk4gyFAYUBMH30+HurREyA== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-minify-gradients@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/postcss-minify-gradients/-/postcss-minify-gradients-5.1.1.tgz#f1fe1b4f498134a5068240c2f25d46fcd236ba2c" + integrity sha512-VGvXMTpCEo4qHTNSa9A0a3D+dxGFZCYwR6Jokk+/3oB6flu2/PnPXAh2x7x52EkY5xlIHLm+Le8tJxe/7TNhzw== + dependencies: + colord "^2.9.1" + cssnano-utils "^3.1.0" + postcss-value-parser "^4.2.0" + +postcss-minify-params@^5.1.3: + version "5.1.3" + resolved "https://registry.npmmirror.com/postcss-minify-params/-/postcss-minify-params-5.1.3.tgz#ac41a6465be2db735099bbd1798d85079a6dc1f9" + integrity sha512-bkzpWcjykkqIujNL+EVEPOlLYi/eZ050oImVtHU7b4lFS82jPnsCb44gvC6pxaNt38Els3jWYDHTjHKf0koTgg== + dependencies: + browserslist "^4.16.6" + cssnano-utils "^3.1.0" + postcss-value-parser "^4.2.0" + +postcss-minify-selectors@^5.2.1: + version "5.2.1" + resolved "https://registry.npmmirror.com/postcss-minify-selectors/-/postcss-minify-selectors-5.2.1.tgz#d4e7e6b46147b8117ea9325a915a801d5fe656c6" + integrity sha512-nPJu7OjZJTsVUmPdm2TcaiohIwxP+v8ha9NehQ2ye9szv4orirRU3SDdtUmKH+10nzn0bAyOXZ0UEr7OpvLehg== + dependencies: + postcss-selector-parser "^6.0.5" + +postcss-modules-extract-imports@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/postcss-modules-extract-imports/-/postcss-modules-extract-imports-3.0.0.tgz#cda1f047c0ae80c97dbe28c3e76a43b88025741d" + integrity sha512-bdHleFnP3kZ4NYDhuGlVK+CMrQ/pqUm8bx/oGL93K6gVwiclvX5x0n76fYMKuIGKzlABOy13zsvqjb0f92TEXw== + +postcss-modules-local-by-default@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.0.tgz#ebbb54fae1598eecfdf691a02b3ff3b390a5a51c" + integrity sha512-sT7ihtmGSF9yhm6ggikHdV0hlziDTX7oFoXtuVWeDd3hHObNkcHRo9V3yg7vCAY7cONyxJC/XXCmmiHHcvX7bQ== + dependencies: + icss-utils "^5.0.0" + postcss-selector-parser "^6.0.2" + postcss-value-parser "^4.1.0" + +postcss-modules-scope@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz#9ef3151456d3bbfa120ca44898dfca6f2fa01f06" + integrity sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg== + dependencies: + postcss-selector-parser "^6.0.4" + +postcss-modules-values@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/postcss-modules-values/-/postcss-modules-values-4.0.0.tgz#d7c5e7e68c3bb3c9b27cbf48ca0bb3ffb4602c9c" + integrity sha512-RDxHkAiEGI78gS2ofyvCsu7iycRv7oqw5xMWn9iMoR0N/7mf9D50ecQqUo5BZ9Zh2vH4bCUR/ktCqbB9m8vJjQ== + dependencies: + icss-utils "^5.0.0" + +postcss-nested@5.0.6: + version "5.0.6" + resolved "https://registry.npmmirror.com/postcss-nested/-/postcss-nested-5.0.6.tgz#466343f7fc8d3d46af3e7dba3fcd47d052a945bc" + integrity sha512-rKqm2Fk0KbA8Vt3AdGN0FB9OBOMDVajMG6ZCf/GoHgdxUJ4sBFp0A/uMIRm+MJUdo33YXEtjqIz8u7DAp8B7DA== + dependencies: + postcss-selector-parser "^6.0.6" + +postcss-normalize-charset@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-charset/-/postcss-normalize-charset-5.1.0.tgz#9302de0b29094b52c259e9b2cf8dc0879879f0ed" + integrity sha512-mSgUJ+pd/ldRGVx26p2wz9dNZ7ji6Pn8VWBajMXFf8jk7vUoSrZ2lt/wZR7DtlZYKesmZI680qjr2CeFF2fbUg== + +postcss-normalize-display-values@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-display-values/-/postcss-normalize-display-values-5.1.0.tgz#72abbae58081960e9edd7200fcf21ab8325c3da8" + integrity sha512-WP4KIM4o2dazQXWmFaqMmcvsKmhdINFblgSeRgn8BJ6vxaMyaJkwAzpPpuvSIoG/rmX3M+IrRZEz2H0glrQNEA== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-normalize-positions@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-positions/-/postcss-normalize-positions-5.1.0.tgz#902a7cb97cf0b9e8b1b654d4a43d451e48966458" + integrity sha512-8gmItgA4H5xiUxgN/3TVvXRoJxkAWLW6f/KKhdsH03atg0cB8ilXnrB5PpSshwVu/dD2ZsRFQcR1OEmSBDAgcQ== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-normalize-repeat-style@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-repeat-style/-/postcss-normalize-repeat-style-5.1.0.tgz#f6d6fd5a54f51a741cc84a37f7459e60ef7a6398" + integrity sha512-IR3uBjc+7mcWGL6CtniKNQ4Rr5fTxwkaDHwMBDGGs1x9IVRkYIT/M4NelZWkAOBdV6v3Z9S46zqaKGlyzHSchw== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-normalize-string@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-string/-/postcss-normalize-string-5.1.0.tgz#411961169e07308c82c1f8c55f3e8a337757e228" + integrity sha512-oYiIJOf4T9T1N4i+abeIc7Vgm/xPCGih4bZz5Nm0/ARVJ7K6xrDlLwvwqOydvyL3RHNf8qZk6vo3aatiw/go3w== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-normalize-timing-functions@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-timing-functions/-/postcss-normalize-timing-functions-5.1.0.tgz#d5614410f8f0b2388e9f240aa6011ba6f52dafbb" + integrity sha512-DOEkzJ4SAXv5xkHl0Wa9cZLF3WCBhF3o1SKVxKQAa+0pYKlueTpCgvkFAHfk+Y64ezX9+nITGrDZeVGgITJXjg== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-normalize-unicode@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-unicode/-/postcss-normalize-unicode-5.1.0.tgz#3d23aede35e160089a285e27bf715de11dc9db75" + integrity sha512-J6M3MizAAZ2dOdSjy2caayJLQT8E8K9XjLce8AUQMwOrCvjCHv24aLC/Lps1R1ylOfol5VIDMaM/Lo9NGlk1SQ== + dependencies: + browserslist "^4.16.6" + postcss-value-parser "^4.2.0" + +postcss-normalize-url@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-normalize-url/-/postcss-normalize-url-5.1.0.tgz#ed9d88ca82e21abef99f743457d3729a042adcdc" + integrity sha512-5upGeDO+PVthOxSmds43ZeMeZfKH+/DKgGRD7TElkkyS46JXAUhMzIKiCa7BabPeIy3AQcTkXwVVN7DbqsiCew== + dependencies: + normalize-url "^6.0.1" + postcss-value-parser "^4.2.0" + +postcss-normalize-whitespace@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/postcss-normalize-whitespace/-/postcss-normalize-whitespace-5.1.1.tgz#08a1a0d1ffa17a7cc6efe1e6c9da969cc4493cfa" + integrity sha512-83ZJ4t3NUDETIHTa3uEg6asWjSBYL5EdkVB0sDncx9ERzOKBVJIUeDO9RyA9Zwtig8El1d79HBp0JEi8wvGQnA== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-ordered-values@^5.1.2: + version "5.1.2" + resolved "https://registry.npmmirror.com/postcss-ordered-values/-/postcss-ordered-values-5.1.2.tgz#daffacd4abf327d52d5ac570b59dfbcf4b836614" + integrity sha512-wr2avRbW4HS2XE2ZCqpfp4N/tDC6GZKZ+SVP8UBTOVS8QWrc4TD8MYrebJrvVVlGPKszmiSCzue43NDiVtgDmg== + dependencies: + cssnano-utils "^3.1.0" + postcss-value-parser "^4.2.0" + +postcss-reduce-initial@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-reduce-initial/-/postcss-reduce-initial-5.1.0.tgz#fc31659ea6e85c492fb2a7b545370c215822c5d6" + integrity sha512-5OgTUviz0aeH6MtBjHfbr57tml13PuedK/Ecg8szzd4XRMbYxH4572JFG067z+FqBIf6Zp/d+0581glkvvWMFw== + dependencies: + browserslist "^4.16.6" + caniuse-api "^3.0.0" + +postcss-reduce-transforms@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-reduce-transforms/-/postcss-reduce-transforms-5.1.0.tgz#333b70e7758b802f3dd0ddfe98bb1ccfef96b6e9" + integrity sha512-2fbdbmgir5AvpW9RLtdONx1QoYG2/EtqpNQbFASDlixBbAYuTcJ0dECwlqNqH7VbaUnEnh8SrxOe2sRIn24XyQ== + dependencies: + postcss-value-parser "^4.2.0" + +postcss-selector-parser@^6.0.10, postcss-selector-parser@^6.0.2, postcss-selector-parser@^6.0.4, postcss-selector-parser@^6.0.5, postcss-selector-parser@^6.0.6, postcss-selector-parser@^6.0.9: + version "6.0.10" + resolved "https://registry.npmmirror.com/postcss-selector-parser/-/postcss-selector-parser-6.0.10.tgz#79b61e2c0d1bfc2602d549e11d0876256f8df88d" + integrity sha512-IQ7TZdoaqbT+LCpShg46jnZVlhWD2w6iQYAcYXfHARZ7X1t/UGhhceQDs5X0cGqKvYlHNOuv7Oa1xmb0oQuA3w== + dependencies: + cssesc "^3.0.0" + util-deprecate "^1.0.2" + +postcss-svgo@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/postcss-svgo/-/postcss-svgo-5.1.0.tgz#0a317400ced789f233a28826e77523f15857d80d" + integrity sha512-D75KsH1zm5ZrHyxPakAxJWtkyXew5qwS70v56exwvw542d9CRtTo78K0WeFxZB4G7JXKKMbEZtZayTGdIky/eA== + dependencies: + postcss-value-parser "^4.2.0" + svgo "^2.7.0" + +postcss-unique-selectors@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/postcss-unique-selectors/-/postcss-unique-selectors-5.1.1.tgz#a9f273d1eacd09e9aa6088f4b0507b18b1b541b6" + integrity sha512-5JiODlELrz8L2HwxfPnhOWZYWDxVHWL83ufOv84NrcgipI7TaeRsatAhK4Tr2/ZiYldpK/wBvw5BD3qfaK96GA== + dependencies: + postcss-selector-parser "^6.0.5" + +postcss-value-parser@^4.0.0, postcss-value-parser@^4.1.0, postcss-value-parser@^4.2.0: + version "4.2.0" + resolved "https://registry.npmmirror.com/postcss-value-parser/-/postcss-value-parser-4.2.0.tgz#723c09920836ba6d3e5af019f92bc0971c02e514" + integrity sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ== + +postcss@7.0.36: + version "7.0.36" + resolved "https://registry.npmmirror.com/postcss/-/postcss-7.0.36.tgz#056f8cffa939662a8f5905950c07d5285644dfcb" + integrity sha512-BebJSIUMwJHRH0HAQoxN4u1CN86glsrwsW0q7T+/m44eXOUAxSNdHRkNZPYz5vVUbg17hFgOQDE7fZk7li3pZw== + dependencies: + chalk "^2.4.2" + source-map "^0.6.1" + supports-color "^6.1.0" + +postcss@^7.0.27: + version "7.0.39" + resolved "https://registry.npmmirror.com/postcss/-/postcss-7.0.39.tgz#9624375d965630e2e1f2c02a935c82a59cb48309" + integrity sha512-yioayjNbHn6z1/Bywyb2Y4s3yvDAeXGOyxqD+LnVOinq6Mdmd++SW2wUNVzavyyHxd6+DxzWGIuosg6P1Rj8uA== + dependencies: + picocolors "^0.2.1" + source-map "^0.6.1" + +postcss@^8.2.15, postcss@^8.2.9, postcss@^8.3.11, postcss@^8.4.14: + version "8.4.14" + resolved "https://registry.npmmirror.com/postcss/-/postcss-8.4.14.tgz#ee9274d5622b4858c1007a74d76e42e56fd21caf" + integrity sha512-E398TUmfAYFPBSdzgeieK2Y1+1cpdxJx8yXbK/m57nRhKSmk1GB2tO4lbLBtlkfPQTDKfe4Xqv1ASWPpayPEig== + dependencies: + nanoid "^3.3.4" + picocolors "^1.0.0" + source-map-js "^1.0.2" + +prebuild-install@^7.1.0: + version "7.1.1" + resolved "https://registry.npmmirror.com/prebuild-install/-/prebuild-install-7.1.1.tgz#de97d5b34a70a0c81334fd24641f2a1702352e45" + integrity sha512-jAXscXWMcCK8GgCoHOfIr0ODh5ai8mj63L2nWrjuAgXE6tDyYGnx4/8o/rCgU+B4JSyZBKbeZqzhtwtC3ovxjw== + dependencies: + detect-libc "^2.0.0" + expand-template "^2.0.3" + github-from-package "0.0.0" + minimist "^1.2.3" + mkdirp-classic "^0.5.3" + napi-build-utils "^1.0.1" + node-abi "^3.3.0" + pump "^3.0.0" + rc "^1.2.7" + simple-get "^4.0.0" + tar-fs "^2.0.0" + tunnel-agent "^0.6.0" + +prelude-ls@^1.2.1: + version "1.2.1" + resolved "https://registry.npmmirror.com/prelude-ls/-/prelude-ls-1.2.1.tgz#debc6489d7a6e6b0e7611888cec880337d316396" + integrity sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g== + +prepend-http@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/prepend-http/-/prepend-http-2.0.0.tgz#e92434bfa5ea8c19f41cdfd401d741a3c819d897" + integrity sha512-ravE6m9Atw9Z/jjttRUZ+clIXogdghyZAuWJ3qEzjT+jI/dL1ifAqhZeC5VHzQp1MSt1+jxKkFNemj/iO7tVUA== + +prettier@^2.6.2: + version "2.7.0" + resolved "https://registry.npmmirror.com/prettier/-/prettier-2.7.0.tgz#a4fdae07e5596c51c9857ea676cd41a0163879d6" + integrity sha512-nwoX4GMFgxoPC6diHvSwmK/4yU8FFH3V8XWtLQrbj4IBsK2pkYhG4kf/ljF/haaZ/aii+wNJqISrCDPgxGWDVQ== + +pretty-bytes@^5.1.0, pretty-bytes@^5.4.1: + version "5.6.0" + resolved "https://registry.npmmirror.com/pretty-bytes/-/pretty-bytes-5.6.0.tgz#356256f643804773c82f64723fe78c92c62beaeb" + integrity sha512-FFw039TmrBqFK8ma/7OL3sDz/VytdtJr044/QUJtH0wK9lb9jLq9tJyIxUwtQJHwar2BqtiA4iCWSwo9JLkzFg== + +pretty-error@^2.1.2: + version "2.1.2" + resolved "https://registry.npmmirror.com/pretty-error/-/pretty-error-2.1.2.tgz#be89f82d81b1c86ec8fdfbc385045882727f93b6" + integrity sha512-EY5oDzmsX5wvuynAByrmY0P0hcp+QpnAKbJng2A2MPjVKXCxrDSUkzghVJ4ZGPIv+JC4gX8fPUWscC0RtjsWGw== + dependencies: + lodash "^4.17.20" + renderkid "^2.0.4" + +probe-image-size@^7.2.3: + version "7.2.3" + resolved "https://registry.npmmirror.com/probe-image-size/-/probe-image-size-7.2.3.tgz#d49c64be540ec8edea538f6f585f65a9b3ab4309" + integrity sha512-HubhG4Rb2UH8YtV4ba0Vp5bQ7L78RTONYu/ujmCu5nBI8wGv24s4E9xSKBi0N1MowRpxk76pFCpJtW0KPzOK0w== + dependencies: + lodash.merge "^4.6.2" + needle "^2.5.2" + stream-parser "~0.3.1" + +process-nextick-args@~2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/process-nextick-args/-/process-nextick-args-2.0.1.tgz#7820d9b16120cc55ca9ae7792680ae7dba6d7fe2" + integrity sha512-3ouUOpQhtgrbOa17J7+uxOTpITYWaGP7/AhoR3+A+/1e9skrzelGi/dXzEYyvbxubEF6Wn2ypscTKiKJFFn1ag== + +process@^0.11.10: + version "0.11.10" + resolved "https://registry.npmmirror.com/process/-/process-0.11.10.tgz#7332300e840161bda3e69a1d1d91a7d4bc16f182" + integrity sha512-cdGef/drWFoydD1JsMzuFf8100nZl+GT+yacc2bEced5f9Rjk4z+WtFUTBu9PhOi9j/jfmBPu0mMEY4wIdAF8A== + +progress@^2.0.0, progress@^2.0.3: + version "2.0.3" + resolved "https://registry.npmmirror.com/progress/-/progress-2.0.3.tgz#7e8cf8d8f5b8f239c1bc68beb4eb78567d572ef8" + integrity sha512-7PiHtLll5LdnKIMw100I+8xJXR5gW2QwWYkT6iJva0bXitZKa/XMrSbdmg3r2Xnaidz9Qumd0VPaMrZlF9V9sA== + +promise@^7.1.1: + version "7.3.1" + resolved "https://registry.npmmirror.com/promise/-/promise-7.3.1.tgz#064b72602b18f90f29192b8b1bc418ffd1ebd3bf" + integrity sha512-nolQXZ/4L+bP/UGlkfaIujX9BKxGwmQ9OT4mOt5yvy8iK1h3wqTEJCijzGANTCCl9nWjY41juyAn2K3Q1hLLTg== + dependencies: + asap "~2.0.3" + +prompts@^2.4.2: + version "2.4.2" + resolved "https://registry.npmmirror.com/prompts/-/prompts-2.4.2.tgz#7b57e73b3a48029ad10ebd44f74b01722a4cb069" + integrity sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q== + dependencies: + kleur "^3.0.3" + sisteransi "^1.0.5" + +prop-types@^15.6.1, prop-types@^15.7.2, prop-types@^15.8.1: + version "15.8.1" + resolved "https://registry.npmmirror.com/prop-types/-/prop-types-15.8.1.tgz#67d87bf1a694f48435cf332c24af10214a3140b5" + integrity sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg== + dependencies: + loose-envify "^1.4.0" + object-assign "^4.1.1" + react-is "^16.13.1" + +proper-lockfile@^4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/proper-lockfile/-/proper-lockfile-4.1.2.tgz#c8b9de2af6b2f1601067f98e01ac66baa223141f" + integrity sha512-TjNPblN4BwAWMXU8s9AEz4JmQxnD1NNL7bNOY/AKUzyamc379FWASUhc/K1pL2noVb+XmZKLL68cjzLsiOAMaA== + dependencies: + graceful-fs "^4.2.4" + retry "^0.12.0" + signal-exit "^3.0.2" + +property-information@^5.0.0, property-information@^5.3.0: + version "5.6.0" + resolved "https://registry.npmmirror.com/property-information/-/property-information-5.6.0.tgz#61675545fb23002f245c6540ec46077d4da3ed69" + integrity sha512-YUHSPk+A30YPv+0Qf8i9Mbfe/C0hdPXk1s1jPVToV8pk8BQtpw10ct89Eo7OWkutrwqvT0eicAxlOg3dOAu8JA== + dependencies: + xtend "^4.0.0" + +protocols@^1.1.0, protocols@^1.4.0: + version "1.4.8" + resolved "https://registry.npmmirror.com/protocols/-/protocols-1.4.8.tgz#48eea2d8f58d9644a4a32caae5d5db290a075ce8" + integrity sha512-IgjKyaUSjsROSO8/D49Ab7hP8mJgTYcqApOqdPhLoPxAplXmkp+zRvsrSQjFn5by0rhm4VH0GAUELIPpx7B1yg== + +proxy-addr@~2.0.7: + version "2.0.7" + resolved "https://registry.npmmirror.com/proxy-addr/-/proxy-addr-2.0.7.tgz#f19fe69ceab311eeb94b42e70e8c2070f9ba1025" + integrity sha512-llQsMLSUDUPT44jdrU/O37qlnifitDP+ZwrmmZcoSKyLKvtZxpyV0n2/bD/N4tBAAZ/gJEdZU7KMraoK1+XYAg== + dependencies: + forwarded "0.2.0" + ipaddr.js "1.9.1" + +pseudomap@^1.0.1, pseudomap@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/pseudomap/-/pseudomap-1.0.2.tgz#f052a28da70e618917ef0a8ac34c1ae5a68286b3" + integrity sha512-b/YwNhb8lk1Zz2+bXXpS/LK9OisiZZ1SNsSLxN1x2OXVEhW2Ckr/7mWE5vrC1ZTiJlD9g19jWszTmJsB+oEpFQ== + +psl@^1.1.28: + version "1.8.0" + resolved "https://registry.npmmirror.com/psl/-/psl-1.8.0.tgz#9326f8bcfb013adcc005fdff056acce020e51c24" + integrity sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ== + +pump@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/pump/-/pump-3.0.0.tgz#b4a2116815bde2f4e1ea602354e8c75565107a64" + integrity sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww== + dependencies: + end-of-stream "^1.1.0" + once "^1.3.1" + +punycode@^2.1.0, punycode@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec" + integrity sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A== + +pupa@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/pupa/-/pupa-2.1.1.tgz#f5e8fd4afc2c5d97828faa523549ed8744a20d62" + integrity sha512-l1jNAspIBSFqbT+y+5FosojNpVpF94nlI+wDUpqP9enwOTfHx9f0gh5nB96vl+6yTpsJsypeNrwfzPrKuHB41A== + dependencies: + escape-goat "^2.0.0" + +pure-color@^1.2.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/pure-color/-/pure-color-1.3.0.tgz#1fe064fb0ac851f0de61320a8bf796836422f33e" + integrity sha512-QFADYnsVoBMw1srW7OVKEYjG+MbIa49s54w1MA1EDY6r2r/sTcKKYqRX1f4GYvnXP7eN/Pe9HFcX+hwzmrXRHA== + +q@^1.1.2: + version "1.5.1" + resolved "https://registry.npmmirror.com/q/-/q-1.5.1.tgz#7e32f75b41381291d04611f1bf14109ac00651d7" + integrity sha512-kV/CThkXo6xyFEZUugw/+pIOywXcDbFYgSct5cT3gqlbkBE1SJdwy6UQoZvodiWF/ckQLZyDE/Bu1M6gVu5lVw== + +qs@6.10.3: + version "6.10.3" + resolved "https://registry.npmmirror.com/qs/-/qs-6.10.3.tgz#d6cde1b2ffca87b5aa57889816c5f81535e22e8e" + integrity sha512-wr7M2E0OFRfIfJZjKGieI8lBKb7fRCH4Fv5KNPEs7gJ8jadvotdsS08PzOKR7opXhZ/Xkjtt3WF9g38drmyRqQ== + dependencies: + side-channel "^1.0.4" + +qs@^6.9.4: + version "6.10.5" + resolved "https://registry.npmmirror.com/qs/-/qs-6.10.5.tgz#974715920a80ff6a262264acd2c7e6c2a53282b4" + integrity sha512-O5RlPh0VFtR78y79rgcgKK4wbAI0C5zGVLztOIdpWX6ep368q5Hv6XRxDvXuZ9q3C6v+e3n8UfZZJw7IIG27eQ== + dependencies: + side-channel "^1.0.4" + +qs@~6.5.2: + version "6.5.3" + resolved "https://registry.npmmirror.com/qs/-/qs-6.5.3.tgz#3aeeffc91967ef6e35c0e488ef46fb296ab76aad" + integrity sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA== + +query-string@^6.13.8, query-string@^6.14.1: + version "6.14.1" + resolved "https://registry.npmmirror.com/query-string/-/query-string-6.14.1.tgz#7ac2dca46da7f309449ba0f86b1fd28255b0c86a" + integrity sha512-XDxAeVmpfu1/6IjyT/gXHOl+S0vQ9owggJ30hhWKdHAsNPOcasn5o9BW0eejZqL2e4vMjhAxoW3jVHcD6mbcYw== + dependencies: + decode-uri-component "^0.2.0" + filter-obj "^1.1.0" + split-on-first "^1.0.0" + strict-uri-encode "^2.0.0" + +querystring@^0.2.0: + version "0.2.1" + resolved "https://registry.npmmirror.com/querystring/-/querystring-0.2.1.tgz#40d77615bb09d16902a85c3e38aa8b5ed761c2dd" + integrity sha512-wkvS7mL/JMugcup3/rMitHmd9ecIGd2lhFhK9N3UUQ450h66d1r3Y9nvXzQAW1Lq+wyx61k/1pfKS5KuKiyEbg== + +queue-microtask@^1.2.2: + version "1.2.3" + resolved "https://registry.npmmirror.com/queue-microtask/-/queue-microtask-1.2.3.tgz#4929228bbc724dfac43e0efb058caf7b6cfb6243" + integrity sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A== + +quick-lru@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/quick-lru/-/quick-lru-4.0.1.tgz#5b8878f113a58217848c6482026c73e1ba57727f" + integrity sha512-ARhCpm70fzdcvNQfPoy49IaanKkTlRWF2JMzqhcJbhSFRZv7nPTvZJdcY7301IPmvW+/p0RgIWnQDLJxifsQ7g== + +quick-lru@^5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/quick-lru/-/quick-lru-5.1.1.tgz#366493e6b3e42a3a6885e2e99d18f80fb7a8c932" + integrity sha512-WuyALRjWPDGtt/wzJiadO5AXY+8hZ80hVpe6MyivgraREW751X3SbhRvG3eLKOYN+8VEvqLcf3wdnt44Z4S4SA== + +randombytes@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/randombytes/-/randombytes-2.1.0.tgz#df6f84372f0270dc65cdf6291349ab7a473d4f2a" + integrity sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ== + dependencies: + safe-buffer "^5.1.0" + +range-parser@^1.2.1, range-parser@~1.2.1: + version "1.2.1" + resolved "https://registry.npmmirror.com/range-parser/-/range-parser-1.2.1.tgz#3cf37023d199e1c24d1a55b84800c2f3e6468031" + integrity sha512-Hrgsx+orqoygnmhFbKaHE6c296J+HTAQXoxEF6gNupROmmGJRoyzfG3ccAveqCBrwr/2yxQ5BVd/GTl5agOwSg== + +raw-body@2.5.1, raw-body@^2.3.0, raw-body@^2.4.1: + version "2.5.1" + resolved "https://registry.npmmirror.com/raw-body/-/raw-body-2.5.1.tgz#fe1b1628b181b700215e5fd42389f98b71392857" + integrity sha512-qqJBtEyVgS0ZmPGdCFPWJ3FreoqvG4MVQln/kCgF7Olq95IbOp0/BWyMwbdtn4VTvkM8Y7khCQ2Xgk/tcrCXig== + dependencies: + bytes "3.1.2" + http-errors "2.0.0" + iconv-lite "0.4.24" + unpipe "1.0.0" + +raw-loader@^4.0.2: + version "4.0.2" + resolved "https://registry.npmmirror.com/raw-loader/-/raw-loader-4.0.2.tgz#1aac6b7d1ad1501e66efdac1522c73e59a584eb6" + integrity sha512-ZnScIV3ag9A4wPX/ZayxL/jZH+euYb6FcUinPcgiQW0+UBtEv0O6Q3lGd3cqJ+GHH+rksEv3Pj99oxJ3u3VIKA== + dependencies: + loader-utils "^2.0.0" + schema-utils "^3.0.0" + +rc-align@^4.0.0: + version "4.0.12" + resolved "https://registry.npmmirror.com/rc-align/-/rc-align-4.0.12.tgz#065b5c68a1cc92a00800c9239320d9fdf5f16207" + integrity sha512-3DuwSJp8iC/dgHzwreOQl52soj40LchlfUHtgACOUtwGuoFIOVh6n/sCpfqCU8kO5+iz6qR0YKvjgB8iPdE3aQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + dom-align "^1.7.0" + lodash "^4.17.21" + rc-util "^5.3.0" + resize-observer-polyfill "^1.5.1" + +rc-cascader@~3.6.0: + version "3.6.0" + resolved "https://registry.npmmirror.com/rc-cascader/-/rc-cascader-3.6.0.tgz#7db0d373edf4c276bba4b68b7de57fad1486c908" + integrity sha512-p9qwt8E8ZICzPIzyfXF5y7/lbJhRowFj8YhWpdytMomHUZ568duFNwA4H5QVqdC6hg/HIV1YEawOE5jlxSpeww== + dependencies: + "@babel/runtime" "^7.12.5" + array-tree-filter "^2.1.0" + classnames "^2.3.1" + rc-select "~14.1.0" + rc-tree "~5.6.3" + rc-util "^5.6.1" + +rc-checkbox@~2.3.0: + version "2.3.2" + resolved "https://registry.npmmirror.com/rc-checkbox/-/rc-checkbox-2.3.2.tgz#f91b3678c7edb2baa8121c9483c664fa6f0aefc1" + integrity sha512-afVi1FYiGv1U0JlpNH/UaEXdh6WUJjcWokj/nUN2TgG80bfG+MDdbfHKlLcNNba94mbjy2/SXJ1HDgrOkXGAjg== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + +rc-collapse@~3.3.0: + version "3.3.0" + resolved "https://registry.npmmirror.com/rc-collapse/-/rc-collapse-3.3.0.tgz#ecde33a06ca53c6c672c6a46c701052b88723950" + integrity sha512-nkxjhpYAAwEVbBvZ/qoatLecD0PpRtQ5ja9G+FP1QmsWhs/4VCruhjvRdSpMn9vfluKUnePe3PEy8eeqTeuE0g== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-motion "^2.3.4" + rc-util "^5.2.1" + shallowequal "^1.1.0" + +rc-dialog@~8.9.0: + version "8.9.0" + resolved "https://registry.npmmirror.com/rc-dialog/-/rc-dialog-8.9.0.tgz#04dc39522f0321ed2e06018d4a7e02a4c32bd3ea" + integrity sha512-Cp0tbJnrvPchJfnwIvOMWmJ4yjX3HWFatO6oBFD1jx8QkgsQCR0p8nUWAKdd3seLJhEC39/v56kZaEjwp9muoQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.6" + rc-motion "^2.3.0" + rc-util "^5.21.0" + +rc-drawer@~4.4.2: + version "4.4.3" + resolved "https://registry.npmmirror.com/rc-drawer/-/rc-drawer-4.4.3.tgz#2094937a844e55dc9644236a2d9fba79c344e321" + integrity sha512-FYztwRs3uXnFOIf1hLvFxIQP9MiZJA+0w+Os8dfDh/90X7z/HqP/Yg+noLCIeHEbKln1Tqelv8ymCAN24zPcfQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.6" + rc-util "^5.7.0" + +rc-dropdown@~4.0.0: + version "4.0.1" + resolved "https://registry.npmmirror.com/rc-dropdown/-/rc-dropdown-4.0.1.tgz#f65d9d3d89750241057db59d5a75e43cd4576b68" + integrity sha512-OdpXuOcme1rm45cR0Jzgfl1otzmU4vuBVb+etXM8vcaULGokAKVpKlw8p6xzspG7jGd/XxShvq+N3VNEfk/l5g== + dependencies: + "@babel/runtime" "^7.18.3" + classnames "^2.2.6" + rc-trigger "^5.3.1" + rc-util "^5.17.0" + +rc-field-form@~1.26.1: + version "1.26.7" + resolved "https://registry.npmmirror.com/rc-field-form/-/rc-field-form-1.26.7.tgz#1794afa50f844170d8907cc575ab482d65db9926" + integrity sha512-CIb7Gw+DG9R+g4HxaDGYHhOjhjQoU2mGU4y+UM2+KQ3uRz9HrrNgTspGvNynn3UamsYcYcaPWZJmiJ6VklkT/w== + dependencies: + "@babel/runtime" "^7.18.0" + async-validator "^4.1.0" + rc-util "^5.8.0" + +rc-image@~5.7.0: + version "5.7.0" + resolved "https://registry.npmmirror.com/rc-image/-/rc-image-5.7.0.tgz#e1a3b21099feb3fb9bf8ef3ce12c3fc11a8c1148" + integrity sha512-v6dzSgYfYrH4liKmOZKZZO+x21sJ9KPXNinBfkAoQg2Ihcd5QZ+P/JjB7v60X981XTPGjegy8U17Z8VUX4V36g== + dependencies: + "@babel/runtime" "^7.11.2" + classnames "^2.2.6" + rc-dialog "~8.9.0" + rc-util "^5.0.6" + +rc-input-number@~7.3.0: + version "7.3.4" + resolved "https://registry.npmmirror.com/rc-input-number/-/rc-input-number-7.3.4.tgz#674aea98260250287d36e330a7e065b174486e9d" + integrity sha512-W9uqSzuvJUnz8H8vsVY4kx+yK51SsAxNTwr8SNH4G3XqQNocLVmKIibKFRjocnYX1RDHMND9FFbgj2h7E7nvGA== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.5" + rc-util "^5.9.8" + +rc-input@~0.0.1-alpha.5: + version "0.0.1-alpha.7" + resolved "https://registry.npmmirror.com/rc-input/-/rc-input-0.0.1-alpha.7.tgz#53e3f13871275c21d92b51f80b698f389ad45dd3" + integrity sha512-eozaqpCYWSY5LBMwlHgC01GArkVEP+XlJ84OMvdkwUnJBSv83Yxa15pZpn7vACAj84uDC4xOA2CoFdbLuqB08Q== + dependencies: + "@babel/runtime" "^7.11.1" + classnames "^2.2.1" + rc-util "^5.18.1" + +rc-mentions@~1.8.0: + version "1.8.0" + resolved "https://registry.npmmirror.com/rc-mentions/-/rc-mentions-1.8.0.tgz#4c0c41605064303f7aedec47d4d07e0bbfcc2dc3" + integrity sha512-ch7yfMMvx2UXy+EvE4axm0Vp6VlVZ30WLrZtLtV/Eb1ty7rQQRzNzCwAHAMyw6tNKTMs9t9sF68AVjAzQ0rvJw== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.6" + rc-menu "~9.6.0" + rc-textarea "^0.3.0" + rc-trigger "^5.0.4" + rc-util "^5.0.1" + +rc-menu@~9.6.0: + version "9.6.0" + resolved "https://registry.npmmirror.com/rc-menu/-/rc-menu-9.6.0.tgz#3263a729a81ae49cfdadee112e97d3c702922829" + integrity sha512-d26waws42U/rVwW/+rOE2FN9pX6wUc9bDy38vVQYoie6gE85auWIpl5oChGlnW6nE2epnTwUsgWl8ipOPgmnUA== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-motion "^2.4.3" + rc-overflow "^1.2.0" + rc-trigger "^5.1.2" + rc-util "^5.12.0" + shallowequal "^1.1.0" + +rc-motion@^2.0.0, rc-motion@^2.0.1, rc-motion@^2.2.0, rc-motion@^2.3.0, rc-motion@^2.3.4, rc-motion@^2.4.3, rc-motion@^2.4.4, rc-motion@^2.5.1: + version "2.6.0" + resolved "https://registry.npmmirror.com/rc-motion/-/rc-motion-2.6.0.tgz#c60c3e7f15257f55a8cd7794a539f0e2cc751399" + integrity sha512-1MDWA9+i174CZ0SIDenSYm2Wb9YbRkrexjZWR0CUFu7D6f23E8Y0KsTgk9NGOLJsGak5ELZK/Y5lOlf5wQdzbw== + dependencies: + "@babel/runtime" "^7.11.1" + classnames "^2.2.1" + rc-util "^5.21.0" + +rc-notification@~4.6.0: + version "4.6.0" + resolved "https://registry.npmmirror.com/rc-notification/-/rc-notification-4.6.0.tgz#4e76fc2d0568f03cc93ac18c9e20763ebe29fa46" + integrity sha512-xF3MKgIoynzjQAO4lqsoraiFo3UXNYlBfpHs0VWvwF+4pimen9/H1DYLN2mfRWhHovW6gRpla73m2nmyIqAMZQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-motion "^2.2.0" + rc-util "^5.20.1" + +rc-overflow@^1.0.0, rc-overflow@^1.2.0: + version "1.2.6" + resolved "https://registry.npmmirror.com/rc-overflow/-/rc-overflow-1.2.6.tgz#e99fabea04ce4fb13f0dd8835aef4e4cdd4c15a2" + integrity sha512-YqbocgzuQxfq2wZy72vdAgrgzzEuM/5d4gF9TBEodCpXPbUeXGrUXNm1J6G1MSkCU2N0ePIgCEu5qD/0Ldi63Q== + dependencies: + "@babel/runtime" "^7.11.1" + classnames "^2.2.1" + rc-resize-observer "^1.0.0" + rc-util "^5.19.2" + +rc-pagination@~3.1.16: + version "3.1.16" + resolved "https://registry.npmmirror.com/rc-pagination/-/rc-pagination-3.1.16.tgz#b0082108cf027eded18ed61d818d31897c343e81" + integrity sha512-GFcHXJ7XxeJDf9B+ndP4PRDt46maSSgYhiwofBMiIGKIlBhJ0wfu8DMCEvaWJJLpI2u4Gb6zF1dHpiqPFrosPg== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + +rc-picker@~2.6.8: + version "2.6.9" + resolved "https://registry.npmmirror.com/rc-picker/-/rc-picker-2.6.9.tgz#2f2f82c5340adbe3b30875a25e015c120eb88c9c" + integrity sha512-yH3UYXCADf7REtOAB5cwe1cyFKtB0p204RCN8JdZGG4uuSOZ1IPTkk/GJS6HOpxspZeJCLGzzajuQMDwck9dsw== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + date-fns "2.x" + dayjs "1.x" + moment "^2.24.0" + rc-trigger "^5.0.4" + rc-util "^5.4.0" + shallowequal "^1.1.0" + +rc-progress@~3.3.2: + version "3.3.3" + resolved "https://registry.npmmirror.com/rc-progress/-/rc-progress-3.3.3.tgz#eb9bffbacab1534f2542f9f6861ce772254362b1" + integrity sha512-MDVNVHzGanYtRy2KKraEaWeZLri2ZHWIRyaE1a9MQ2MuJ09m+Wxj5cfcaoaR6z5iRpHpA59YeUxAlpML8N4PJw== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.6" + rc-util "^5.16.1" + +rc-rate@~2.9.0: + version "2.9.2" + resolved "https://registry.npmmirror.com/rc-rate/-/rc-rate-2.9.2.tgz#4a58965d1ecf91896ebae01d458b59056df0b4ea" + integrity sha512-SaiZFyN8pe0Fgphv8t3+kidlej+cq/EALkAJAc3A0w0XcPaH2L1aggM8bhe1u6GAGuQNAoFvTLjw4qLPGRKV5g== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.5" + rc-util "^5.0.1" + +rc-resize-observer@^1.0.0, rc-resize-observer@^1.1.0, rc-resize-observer@^1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/rc-resize-observer/-/rc-resize-observer-1.2.0.tgz#9f46052f81cdf03498be35144cb7c53fd282c4c7" + integrity sha512-6W+UzT3PyDM0wVCEHfoW3qTHPTvbdSgiA43buiy8PzmeMnfgnDeb9NjdimMXMl3/TcrvvWl5RRVdp+NqcR47pQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + rc-util "^5.15.0" + resize-observer-polyfill "^1.5.1" + +rc-segmented@~2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/rc-segmented/-/rc-segmented-2.1.0.tgz#0e0afe646c1a0e44a0e18785f518c42633ec8efc" + integrity sha512-hUlonro+pYoZcwrH6Vm56B2ftLfQh046hrwif/VwLIw1j3zGt52p5mREBwmeVzXnSwgnagpOpfafspzs1asjGw== + dependencies: + "@babel/runtime" "^7.11.1" + classnames "^2.2.1" + rc-motion "^2.4.4" + rc-util "^5.17.0" + +rc-select@~14.1.0, rc-select@~14.1.1: + version "14.1.5" + resolved "https://registry.npmmirror.com/rc-select/-/rc-select-14.1.5.tgz#9e40dfdfd207d1d90c93121d86c6fbeb4ec0281c" + integrity sha512-CvcmylICKSrPWCJMgGiHqozVhco9kJpQSj/x5wqLN9JStpDFD1oMNYiJYfkMjQ1LxZkN/eZpL1D2KUXJhXd8rw== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-motion "^2.0.1" + rc-overflow "^1.0.0" + rc-trigger "^5.0.4" + rc-util "^5.16.1" + rc-virtual-list "^3.2.0" + +rc-slider@~10.0.0: + version "10.0.0" + resolved "https://registry.npmmirror.com/rc-slider/-/rc-slider-10.0.0.tgz#8ffe1dd3c8799c9d1f81ac808976f18af3dca206" + integrity sha512-Bk54UIKWW4wyhHcL8ehAxt+wX+n69dscnHTX6Uv0FMxSke/TGrlkZz1LSIWblCpfE2zr/dwR2Ca8nZGk3U+Tbg== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.5" + rc-tooltip "^5.0.1" + rc-util "^5.18.1" + shallowequal "^1.1.0" + +rc-steps@~4.1.0: + version "4.1.4" + resolved "https://registry.npmmirror.com/rc-steps/-/rc-steps-4.1.4.tgz#0ba82db202d59ca52d0693dc9880dd145b19dc23" + integrity sha512-qoCqKZWSpkh/b03ASGx1WhpKnuZcRWmvuW+ZUu4mvMdfvFzVxblTwUM+9aBd0mlEUFmt6GW8FXhMpHkK3Uzp3w== + dependencies: + "@babel/runtime" "^7.10.2" + classnames "^2.2.3" + rc-util "^5.0.1" + +rc-switch@~3.2.0: + version "3.2.2" + resolved "https://registry.npmmirror.com/rc-switch/-/rc-switch-3.2.2.tgz#d001f77f12664d52595b4f6fb425dd9e66fba8e8" + integrity sha512-+gUJClsZZzvAHGy1vZfnwySxj+MjLlGRyXKXScrtCTcmiYNPzxDFOxdQ/3pK1Kt/0POvwJ/6ALOR8gwdXGhs+A== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + rc-util "^5.0.1" + +rc-table@~7.24.0: + version "7.24.2" + resolved "https://registry.npmmirror.com/rc-table/-/rc-table-7.24.2.tgz#fbccf5ef4b84cdb38c8a0b416365de157483bf51" + integrity sha512-yefqhtc4V3BeWG2bnDhWYxWX1MOckvW2KU1J55pntZmIGrov5Hx8tQn2gcs6OM0fJ6NgEwUvVEknsCsWI24zUg== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.5" + rc-resize-observer "^1.1.0" + rc-util "^5.14.0" + shallowequal "^1.1.0" + +rc-tabs@~11.16.0: + version "11.16.0" + resolved "https://registry.npmmirror.com/rc-tabs/-/rc-tabs-11.16.0.tgz#12447069ea1dc480c729e1e40661cfbd46ac4efe" + integrity sha512-CIDPv3lHaXSHTJevmFP2eHoD3Hq9psfKbOZYf6D4FYPACloNGHpz44y3RGeJgataQ7omFLrGBm3dOBMUki87tA== + dependencies: + "@babel/runtime" "^7.11.2" + classnames "2.x" + rc-dropdown "~4.0.0" + rc-menu "~9.6.0" + rc-resize-observer "^1.0.0" + rc-util "^5.5.0" + +rc-textarea@^0.3.0, rc-textarea@~0.3.0: + version "0.3.7" + resolved "https://registry.npmmirror.com/rc-textarea/-/rc-textarea-0.3.7.tgz#987142891efdedb774883c07e2f51b318fde5a11" + integrity sha512-yCdZ6binKmAQB13hc/oehh0E/QRwoPP1pjF21aHBxlgXO3RzPF6dUu4LG2R4FZ1zx/fQd2L1faktulrXOM/2rw== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "^2.2.1" + rc-resize-observer "^1.0.0" + rc-util "^5.7.0" + shallowequal "^1.1.0" + +rc-tooltip@^5.0.1, rc-tooltip@~5.1.1: + version "5.1.1" + resolved "https://registry.npmmirror.com/rc-tooltip/-/rc-tooltip-5.1.1.tgz#94178ed162d0252bc4993b725f5dc2ac0fccf154" + integrity sha512-alt8eGMJulio6+4/uDm7nvV+rJq9bsfxFDCI0ljPdbuoygUscbsMYb6EQgwib/uqsXQUvzk+S7A59uYHmEgmDA== + dependencies: + "@babel/runtime" "^7.11.2" + rc-trigger "^5.0.0" + +rc-tree-select@~5.4.0: + version "5.4.0" + resolved "https://registry.npmmirror.com/rc-tree-select/-/rc-tree-select-5.4.0.tgz#c94b961aca68689f5ee3a43e33881cf693d195ef" + integrity sha512-reRbOqC7Ic/nQocJAJeCl4n6nJUY3NoqiwRXKvhjgZJU7NGr9vIccXEsY+Lghkw5UMpPoxGsIJB0jiAvM18XYA== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-select "~14.1.0" + rc-tree "~5.6.1" + rc-util "^5.16.1" + +rc-tree@~5.6.1, rc-tree@~5.6.3, rc-tree@~5.6.5: + version "5.6.5" + resolved "https://registry.npmmirror.com/rc-tree/-/rc-tree-5.6.5.tgz#1947337fc48f3fe20fabaafb1aed3e4ff1ce71b4" + integrity sha512-Bnyen46B251APyRZ9D/jYeTnSqbSEvK2AkU5B4vWkNYgUJNPrxO+VMgcDRedP/8N7YcsgdDT9hxqVvNOq7oCAQ== + dependencies: + "@babel/runtime" "^7.10.1" + classnames "2.x" + rc-motion "^2.0.1" + rc-util "^5.16.1" + rc-virtual-list "^3.4.8" + +rc-trigger@^5.0.0, rc-trigger@^5.0.4, rc-trigger@^5.1.2, rc-trigger@^5.2.10, rc-trigger@^5.3.1: + version "5.3.1" + resolved "https://registry.npmmirror.com/rc-trigger/-/rc-trigger-5.3.1.tgz#acafadf3eaf384e7f466c303bfa0f34c8137d7b8" + integrity sha512-5gaFbDkYSefZ14j2AdzucXzlWgU2ri5uEjkHvsf1ynRhdJbKxNOnw4PBZ9+FVULNGFiDzzlVF8RJnR9P/xrnKQ== + dependencies: + "@babel/runtime" "^7.18.3" + classnames "^2.2.6" + rc-align "^4.0.0" + rc-motion "^2.0.0" + rc-util "^5.19.2" + +rc-upload@~4.3.0: + version "4.3.4" + resolved "https://registry.npmmirror.com/rc-upload/-/rc-upload-4.3.4.tgz#83ff7d3867631c37adbfd72ea3d1fd7e97ca84af" + integrity sha512-uVbtHFGNjHG/RyAfm9fluXB6pvArAGyAx8z7XzXXyorEgVIWj6mOlriuDm0XowDHYz4ycNK0nE0oP3cbFnzxiQ== + dependencies: + "@babel/runtime" "^7.18.3" + classnames "^2.2.5" + rc-util "^5.2.0" + +rc-util@^5.0.1, rc-util@^5.0.6, rc-util@^5.12.0, rc-util@^5.14.0, rc-util@^5.15.0, rc-util@^5.16.1, rc-util@^5.17.0, rc-util@^5.18.1, rc-util@^5.19.2, rc-util@^5.2.0, rc-util@^5.2.1, rc-util@^5.20.0, rc-util@^5.20.1, rc-util@^5.21.0, rc-util@^5.3.0, rc-util@^5.4.0, rc-util@^5.5.0, rc-util@^5.6.1, rc-util@^5.7.0, rc-util@^5.8.0, rc-util@^5.9.4, rc-util@^5.9.8: + version "5.21.5" + resolved "https://registry.npmmirror.com/rc-util/-/rc-util-5.21.5.tgz#6e2a5699f820ba915f43f11a4b7dfb0b0672d0fa" + integrity sha512-ip7HqX37Cy/RDl9MlrFp+FbcKnsWZ22sF5MS5eSpYLtg5MpC0TMqGb5ukBatoOhgjnLL+eJGR6e7YAJ/dhK09A== + dependencies: + "@babel/runtime" "^7.18.3" + react-is "^16.12.0" + shallowequal "^1.1.0" + +rc-virtual-list@^3.2.0, rc-virtual-list@^3.4.8: + version "3.4.8" + resolved "https://registry.npmmirror.com/rc-virtual-list/-/rc-virtual-list-3.4.8.tgz#c24c10c6940546b7e2a5e9809402c6716adfd26c" + integrity sha512-qSN+Rv4i/E7RCTvTMr1uZo7f3crJJg/5DekoCagydo9zsXrxj07zsFSxqizqW+ldGA16lwa8So/bIbV9Ofjddg== + dependencies: + classnames "^2.2.6" + rc-resize-observer "^1.0.0" + rc-util "^5.15.0" + +rc@^1.2.7, rc@^1.2.8: + version "1.2.8" + resolved "https://registry.npmmirror.com/rc/-/rc-1.2.8.tgz#cd924bf5200a075b83c188cd6b9e211b7fc0d3ed" + integrity sha512-y3bGgqKj3QBdxLbLkomlohkvsA8gdAiUQlSBJnBhfn+BPxg4bc62d8TcBW15wavDfgexCgccckhcZvywyQYPOw== + dependencies: + deep-extend "^0.6.0" + ini "~1.3.0" + minimist "^1.2.0" + strip-json-comments "~2.0.1" + +react-base16-styling@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/react-base16-styling/-/react-base16-styling-0.6.0.tgz#ef2156d66cf4139695c8a167886cb69ea660792c" + integrity sha512-yvh/7CArceR/jNATXOKDlvTnPKPmGZz7zsenQ3jUwLzHkNUR0CvY3yGYJbWJ/nnxsL8Sgmt5cO3/SILVuPO6TQ== + dependencies: + base16 "^1.0.0" + lodash.curry "^4.0.1" + lodash.flow "^3.3.0" + pure-color "^1.2.0" + +react-dev-utils@^12.0.1: + version "12.0.1" + resolved "https://registry.npmmirror.com/react-dev-utils/-/react-dev-utils-12.0.1.tgz#ba92edb4a1f379bd46ccd6bcd4e7bc398df33e73" + integrity sha512-84Ivxmr17KjUupyqzFode6xKhjwuEJDROWKJy/BthkL7Wn6NJ8h4WE6k/exAv6ImS+0oZLRRW5j/aINMHyeGeQ== + dependencies: + "@babel/code-frame" "^7.16.0" + address "^1.1.2" + browserslist "^4.18.1" + chalk "^4.1.2" + cross-spawn "^7.0.3" + detect-port-alt "^1.1.6" + escape-string-regexp "^4.0.0" + filesize "^8.0.6" + find-up "^5.0.0" + fork-ts-checker-webpack-plugin "^6.5.0" + global-modules "^2.0.0" + globby "^11.0.4" + gzip-size "^6.0.0" + immer "^9.0.7" + is-root "^2.1.0" + loader-utils "^3.2.0" + open "^8.4.0" + pkg-up "^3.1.0" + prompts "^2.4.2" + react-error-overlay "^6.0.11" + recursive-readdir "^2.2.2" + shell-quote "^1.7.3" + strip-ansi "^6.0.1" + text-table "^0.2.0" + +react-dom@^18.1.0: + version "18.2.0" + resolved "https://registry.npmmirror.com/react-dom/-/react-dom-18.2.0.tgz#22aaf38708db2674ed9ada224ca4aa708d821e3d" + integrity sha512-6IMTriUmvsjHUjNtEDudZfuDQUoWXVxKHhlEGSk81n4YFS+r/Kl99wXiwlVXtPBtJenozv2P+hxDsw9eA7Xo6g== + dependencies: + loose-envify "^1.1.0" + scheduler "^0.23.0" + +react-error-overlay@6.0.9: + version "6.0.9" + resolved "https://registry.npmmirror.com/react-error-overlay/-/react-error-overlay-6.0.9.tgz#3c743010c9359608c375ecd6bc76f35d93995b0a" + integrity sha512-nQTTcUu+ATDbrSD1BZHr5kgSD4oF8OFjxun8uAaL8RwPBacGBNPf/yAuVVdx17N8XNzRDMrZ9XcKZHCjPW+9ew== + +react-error-overlay@^6.0.11: + version "6.0.11" + resolved "https://registry.npmmirror.com/react-error-overlay/-/react-error-overlay-6.0.11.tgz#92835de5841c5cf08ba00ddd2d677b6d17ff9adb" + integrity sha512-/6UZ2qgEyH2aqzYZgQPxEnz33NJ2gNsnHA2o5+o4wW9bLM/JYQitNP9xPhsXwC08hMMovfGe/8retsdDsczPRg== + +react-fast-compare@^3.0.1, react-fast-compare@^3.1.1: + version "3.2.0" + resolved "https://registry.npmmirror.com/react-fast-compare/-/react-fast-compare-3.2.0.tgz#641a9da81b6a6320f270e89724fb45a0b39e43bb" + integrity sha512-rtGImPZ0YyLrscKI9xTpV8psd6I8VAtjKCzQDlzyDvqJA8XOW78TXYQwNRNd8g8JZnDu8q9Fu/1v4HPAVwVdHA== + +react-helmet@^6.1.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/react-helmet/-/react-helmet-6.1.0.tgz#a750d5165cb13cf213e44747502652e794468726" + integrity sha512-4uMzEY9nlDlgxr61NL3XbKRy1hEkXmKNXhjbAIOVw5vcFrsdYbH2FEwcNyWvWinl103nXgzYNlns9ca+8kFiWw== + dependencies: + object-assign "^4.1.1" + prop-types "^15.7.2" + react-fast-compare "^3.1.1" + react-side-effect "^2.1.0" + +react-highlight@^0.14.0: + version "0.14.0" + resolved "https://registry.npmmirror.com/react-highlight/-/react-highlight-0.14.0.tgz#5aefa5518baa580f96b68d48129d7a5d2dc0c9ef" + integrity sha512-kWE+KXOXidS7SABhVopOgMnowbI3RAfeGZbnrduLNlWrYAED8sycL9l/Fvw3w0PFpIIawB7mRDnyhDcM/cIIGA== + dependencies: + highlight.js "^10.5.0" + +react-is@^16.12.0, react-is@^16.13.1, react-is@^16.7.0: + version "16.13.1" + resolved "https://registry.npmmirror.com/react-is/-/react-is-16.13.1.tgz#789729a4dc36de2999dc156dd6c1d9c18cea56a4" + integrity sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ== + +react-json-view@^1.21.3: + version "1.21.3" + resolved "https://registry.npmmirror.com/react-json-view/-/react-json-view-1.21.3.tgz#f184209ee8f1bf374fb0c41b0813cff54549c475" + integrity sha512-13p8IREj9/x/Ye4WI/JpjhoIwuzEgUAtgJZNBJckfzJt1qyh24BdTm6UQNGnyTq9dapQdrqvquZTo3dz1X6Cjw== + dependencies: + flux "^4.0.1" + react-base16-styling "^0.6.0" + react-lifecycles-compat "^3.0.4" + react-textarea-autosize "^8.3.2" + +react-lifecycles-compat@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz#4f1a273afdfc8f3488a8c516bfda78f872352362" + integrity sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA== + +react-popper@^2.2.5: + version "2.3.0" + resolved "https://registry.npmmirror.com/react-popper/-/react-popper-2.3.0.tgz#17891c620e1320dce318bad9fede46a5f71c70ba" + integrity sha512-e1hj8lL3uM+sgSR4Lxzn5h1GxBlpa4CQz0XLF8kx4MDrDRWY0Ena4c97PUeSX9i5W3UAfDP0z0FXCTQkoXUl3Q== + dependencies: + react-fast-compare "^3.0.1" + warning "^4.0.2" + +react-property@2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/react-property/-/react-property-2.0.0.tgz#2156ba9d85fa4741faf1918b38efc1eae3c6a136" + integrity sha512-kzmNjIgU32mO4mmH5+iUyrqlpFQhF8K2k7eZ4fdLSOPFrD1XgEuSBv9LDEgxRXTMBqMd8ppT0x6TIzqE5pdGdw== + +react-refresh@^0.9.0: + version "0.9.0" + resolved "https://registry.npmmirror.com/react-refresh/-/react-refresh-0.9.0.tgz#71863337adc3e5c2f8a6bfddd12ae3bfe32aafbf" + integrity sha512-Gvzk7OZpiqKSkxsQvO/mbTN1poglhmAV7gR/DdIrRrSMXraRQQlfikRJOr3Nb9GTMPC5kof948Zy6jJZIFtDvQ== + +react-side-effect@^2.1.0: + version "2.1.1" + resolved "https://registry.npmmirror.com/react-side-effect/-/react-side-effect-2.1.1.tgz#66c5701c3e7560ab4822a4ee2742dee215d72eb3" + integrity sha512-2FoTQzRNTncBVtnzxFOk2mCpcfxQpenBMbk5kSVBg5UcPqV9fRbgY2zhb7GTWWOlpFmAxhClBDlIq8Rsubz1yQ== + +react-textarea-autosize@^8.3.2: + version "8.3.4" + resolved "https://registry.npmmirror.com/react-textarea-autosize/-/react-textarea-autosize-8.3.4.tgz#270a343de7ad350534141b02c9cb78903e553524" + integrity sha512-CdtmP8Dc19xL8/R6sWvtknD/eCXkQr30dtvC4VmGInhRsfF8X/ihXCq6+9l9qbxmKRiq407/7z5fxE7cVWQNgQ== + dependencies: + "@babel/runtime" "^7.10.2" + use-composed-ref "^1.3.0" + use-latest "^1.2.1" + +react@^18.1.0: + version "18.2.0" + resolved "https://registry.npmmirror.com/react/-/react-18.2.0.tgz#555bd98592883255fa00de14f1151a917b5d77d5" + integrity sha512-/3IjMdb2L9QbBdWiW5e3P2/npwMBaU9mHCSCUzNln0ZCYbcfTsGbTJrU/kGemdH2IWmB2ioZ+zkxtmq6g09fGQ== + dependencies: + loose-envify "^1.1.0" + +read-cache@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/read-cache/-/read-cache-1.0.0.tgz#e664ef31161166c9751cdbe8dbcf86b5fb58f774" + integrity sha512-Owdv/Ft7IjOgm/i0xvNDZ1LrRANRfew4b2prF3OWMQLxLfu3bS8FVhCsrSCMK4lR56Y9ya+AThoTpDCTxCmpRA== + dependencies: + pify "^2.3.0" + +read-pkg-up@^7.0.1: + version "7.0.1" + resolved "https://registry.npmmirror.com/read-pkg-up/-/read-pkg-up-7.0.1.tgz#f3a6135758459733ae2b95638056e1854e7ef507" + integrity sha512-zK0TB7Xd6JpCLmlLmufqykGE+/TlOePD6qKClNW7hHDKFh/J7/7gCWGR7joEQEW1bKq3a3yUZSObOoWLFQ4ohg== + dependencies: + find-up "^4.1.0" + read-pkg "^5.2.0" + type-fest "^0.8.1" + +read-pkg@^5.2.0: + version "5.2.0" + resolved "https://registry.npmmirror.com/read-pkg/-/read-pkg-5.2.0.tgz#7bf295438ca5a33e56cd30e053b34ee7250c93cc" + integrity sha512-Ug69mNOpfvKDAc2Q8DRpMjjzdtrnv9HcSMX+4VsZxD1aZ6ZzrIE7rlzXBtWTyhULSMKg076AW6WR5iZpD0JiOg== + dependencies: + "@types/normalize-package-data" "^2.4.0" + normalize-package-data "^2.5.0" + parse-json "^5.0.0" + type-fest "^0.6.0" + +read@^1.0.7: + version "1.0.7" + resolved "https://registry.npmmirror.com/read/-/read-1.0.7.tgz#b3da19bd052431a97671d44a42634adf710b40c4" + integrity sha512-rSOKNYUmaxy0om1BNjMN4ezNT6VKK+2xF4GBhc81mkH7L60i6dp8qPYrkndNLT3QPphoII3maL9PVC9XmhHwVQ== + dependencies: + mute-stream "~0.0.4" + +readable-stream@1.1.x: + version "1.1.14" + resolved "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.1.14.tgz#7cf4c54ef648e3813084c636dd2079e166c081d9" + integrity sha512-+MeVjFf4L44XUkhM1eYbD8fyEsxcV81pqMSR5gblfcLCHfZvbrqy4/qYHE+/R5HoBUT11WV5O08Cr1n3YXkWVQ== + dependencies: + core-util-is "~1.0.0" + inherits "~2.0.1" + isarray "0.0.1" + string_decoder "~0.10.x" + +readable-stream@^2.0.1, readable-stream@^2.0.2, readable-stream@^2.0.6, readable-stream@^2.2.2: + version "2.3.7" + resolved "https://registry.npmmirror.com/readable-stream/-/readable-stream-2.3.7.tgz#1eca1cf711aef814c04f62252a36a62f6cb23b57" + integrity sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw== + dependencies: + core-util-is "~1.0.0" + inherits "~2.0.3" + isarray "~1.0.0" + process-nextick-args "~2.0.0" + safe-buffer "~5.1.1" + string_decoder "~1.1.1" + util-deprecate "~1.0.1" + +readable-stream@^3.1.1, readable-stream@^3.4.0, readable-stream@^3.6.0: + version "3.6.0" + resolved "https://registry.npmmirror.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@~1.0.17, readable-stream@~1.0.27-1: + version "1.0.34" + resolved "https://registry.npmmirror.com/readable-stream/-/readable-stream-1.0.34.tgz#125820e34bc842d2f2aaafafe4c2916ee32c157c" + integrity sha512-ok1qVCJuRkNmvebYikljxJA/UEsKwLl2nI1OmaqAu4/UE+h0wKCHok4XkL/gvi39OacXvw59RJUOFUkDib2rHg== + dependencies: + core-util-is "~1.0.0" + inherits "~2.0.1" + isarray "0.0.1" + string_decoder "~0.10.x" + +readable-web-to-node-stream@^3.0.0: + version "3.0.2" + resolved "https://registry.npmmirror.com/readable-web-to-node-stream/-/readable-web-to-node-stream-3.0.2.tgz#5d52bb5df7b54861fd48d015e93a2cb87b3ee0bb" + integrity sha512-ePeK6cc1EcKLEhJFt/AebMCLL+GgSKhuygrZ/GLaKZYEecIgIECf4UaUuaByiGtzckwR4ain9VzUh95T1exYGw== + dependencies: + readable-stream "^3.6.0" + +readdirp@~3.6.0: + version "3.6.0" + resolved "https://registry.npmmirror.com/readdirp/-/readdirp-3.6.0.tgz#74a370bd857116e245b29cc97340cd431a02a6c7" + integrity sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA== + dependencies: + picomatch "^2.2.1" + +recursive-readdir@^2.2.2: + version "2.2.2" + resolved "https://registry.npmmirror.com/recursive-readdir/-/recursive-readdir-2.2.2.tgz#9946fb3274e1628de6e36b2f6714953b4845094f" + integrity sha512-nRCcW9Sj7NuZwa2XvH9co8NPeXUBhZP7CRKJtU+cS6PW9FpCIFoI5ib0NT1ZrbNuPoRy0ylyCaUL8Gih4LSyFg== + dependencies: + minimatch "3.0.4" + +redent@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/redent/-/redent-3.0.0.tgz#e557b7998316bb53c9f1f56fa626352c6963059f" + integrity sha512-6tDA8g98We0zd0GvVeMT9arEOnTw9qM03L9cJXaCjrip1OO764RDBLBfrB4cwzNGDj5OA5ioymC9GkizgWJDUg== + dependencies: + indent-string "^4.0.0" + strip-indent "^3.0.0" + +redux-thunk@^2.4.0: + version "2.4.1" + resolved "https://registry.npmmirror.com/redux-thunk/-/redux-thunk-2.4.1.tgz#0dd8042cf47868f4b29699941de03c9301a75714" + integrity sha512-OOYGNY5Jy2TWvTL1KgAlVy6dcx3siPJ1wTq741EPyUKfn6W6nChdICjZwCd0p8AZBs5kWpZlbkXW2nE/zjUa+Q== + +redux@4.1.2: + version "4.1.2" + resolved "https://registry.npmmirror.com/redux/-/redux-4.1.2.tgz#140f35426d99bb4729af760afcf79eaaac407104" + integrity sha512-SH8PglcebESbd/shgf6mii6EIoRM0zrQyjcuQ+ojmfxjTtE0z9Y8pa62iA/OJ58qjP6j27uyW4kUF4jl/jd6sw== + dependencies: + "@babel/runtime" "^7.9.2" + +regenerate-unicode-properties@^10.0.1: + version "10.0.1" + resolved "https://registry.npmmirror.com/regenerate-unicode-properties/-/regenerate-unicode-properties-10.0.1.tgz#7f442732aa7934a3740c779bb9b3340dccc1fb56" + integrity sha512-vn5DU6yg6h8hP/2OkQo3K7uVILvY4iu0oI4t3HFa81UPkhGJwkRwM10JEc3upjdhHjs/k8GJY1sRBhk5sr69Bw== + dependencies: + regenerate "^1.4.2" + +regenerate@^1.4.2: + version "1.4.2" + resolved "https://registry.npmmirror.com/regenerate/-/regenerate-1.4.2.tgz#b9346d8827e8f5a32f7ba29637d398b69014848a" + integrity sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A== + +regenerator-runtime@^0.11.0: + version "0.11.1" + resolved "https://registry.npmmirror.com/regenerator-runtime/-/regenerator-runtime-0.11.1.tgz#be05ad7f9bf7d22e056f9726cee5017fbf19e2e9" + integrity sha512-MguG95oij0fC3QV3URf4V2SDYGJhJnJGqvIIgdECeODCT98wSWDAJ94SSuVpYQUoTcGUIL6L4yNB7j1DFFHSBg== + +regenerator-runtime@^0.13.3, regenerator-runtime@^0.13.4, regenerator-runtime@^0.13.7: + version "0.13.9" + resolved "https://registry.npmmirror.com/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz#8925742a98ffd90814988d7566ad30ca3b263b52" + integrity sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA== + +regenerator-transform@^0.15.0: + version "0.15.0" + resolved "https://registry.npmmirror.com/regenerator-transform/-/regenerator-transform-0.15.0.tgz#cbd9ead5d77fae1a48d957cf889ad0586adb6537" + integrity sha512-LsrGtPmbYg19bcPHwdtmXwbW+TqNvtY4riE3P83foeHRroMbH6/2ddFBfab3t7kbzc7v7p4wbkIecHImqt0QNg== + dependencies: + "@babel/runtime" "^7.8.4" + +regex-parser@^2.2.11: + version "2.2.11" + resolved "https://registry.npmmirror.com/regex-parser/-/regex-parser-2.2.11.tgz#3b37ec9049e19479806e878cabe7c1ca83ccfe58" + integrity sha512-jbD/FT0+9MBU2XAZluI7w2OBs1RBi6p9M83nkoZayQXXU9e8Robt69FcZc7wU4eJD/YFTjn1JdCk3rbMJajz8Q== + +regexp.prototype.flags@^1.4.1, regexp.prototype.flags@^1.4.3: + version "1.4.3" + resolved "https://registry.npmmirror.com/regexp.prototype.flags/-/regexp.prototype.flags-1.4.3.tgz#87cab30f80f66660181a3bb7bf5981a872b367ac" + integrity sha512-fjggEOO3slI6Wvgjwflkc4NFRCTZAu5CnNfBd5qOMYhWdn67nJBBu34/TkD++eeFmd8C9r9jfXJ27+nSiRkSUA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + functions-have-names "^1.2.2" + +regexpp@^3.1.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/regexpp/-/regexpp-3.2.0.tgz#0425a2768d8f23bad70ca4b90461fa2f1213e1b2" + integrity sha512-pq2bWo9mVD43nbts2wGv17XLiNLya+GklZ8kaDLV2Z08gDCsGpnKn9BFMepvWuHCbyVvY7J5o5+BVvoQbmlJLg== + +regexpu-core@^5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/regexpu-core/-/regexpu-core-5.0.1.tgz#c531122a7840de743dcf9c83e923b5560323ced3" + integrity sha512-CriEZlrKK9VJw/xQGJpQM5rY88BtuL8DM+AEwvcThHilbxiTAy8vq4iJnd2tqq8wLmjbGZzP7ZcKFjbGkmEFrw== + dependencies: + regenerate "^1.4.2" + regenerate-unicode-properties "^10.0.1" + regjsgen "^0.6.0" + regjsparser "^0.8.2" + unicode-match-property-ecmascript "^2.0.0" + unicode-match-property-value-ecmascript "^2.0.0" + +registry-auth-token@^4.0.0: + version "4.2.1" + resolved "https://registry.npmmirror.com/registry-auth-token/-/registry-auth-token-4.2.1.tgz#6d7b4006441918972ccd5fedcd41dc322c79b250" + integrity sha512-6gkSb4U6aWJB4SF2ZvLb76yCBjcvufXBqvvEx1HbmKPkutswjW1xNVRY0+daljIYRbogN7O0etYSlbiaEQyMyw== + dependencies: + rc "^1.2.8" + +registry-url@^5.0.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/registry-url/-/registry-url-5.1.0.tgz#e98334b50d5434b81136b44ec638d9c2009c5009" + integrity sha512-8acYXXTI0AkQv6RAOjE3vOaIXZkT9wo4LOFbBKYQEEnnMNBpKqdUrI6S4NT0KPIo/WVvJ5tE/X5LF/TQUf0ekw== + dependencies: + rc "^1.2.8" + +regjsgen@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/regjsgen/-/regjsgen-0.6.0.tgz#83414c5354afd7d6627b16af5f10f41c4e71808d" + integrity sha512-ozE883Uigtqj3bx7OhL1KNbCzGyW2NQZPl6Hs09WTvCuZD5sTI4JY58bkbQWa/Y9hxIsvJ3M8Nbf7j54IqeZbA== + +regjsparser@^0.8.2: + version "0.8.4" + resolved "https://registry.npmmirror.com/regjsparser/-/regjsparser-0.8.4.tgz#8a14285ffcc5de78c5b95d62bbf413b6bc132d5f" + integrity sha512-J3LABycON/VNEu3abOviqGHuB/LOtOQj8SKmfP9anY5GfAVw/SPjwzSjxGjbZXIxbGfqTHtJw58C2Li/WkStmA== + dependencies: + jsesc "~0.5.0" + +relay-runtime@12.0.0: + version "12.0.0" + resolved "https://registry.npmmirror.com/relay-runtime/-/relay-runtime-12.0.0.tgz#1e039282bdb5e0c1b9a7dc7f6b9a09d4f4ff8237" + integrity sha512-QU6JKr1tMsry22DXNy9Whsq5rmvwr3LSZiiWV/9+DFpuTWvp+WFhobWMc8TC4OjKFfNhEZy7mOiqUAn5atQtug== + dependencies: + "@babel/runtime" "^7.0.0" + fbjs "^3.0.0" + invariant "^2.2.4" + +remark-footnotes@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/remark-footnotes/-/remark-footnotes-3.0.0.tgz#5756b56f8464fa7ed80dbba0c966136305d8cb8d" + integrity sha512-ZssAvH9FjGYlJ/PBVKdSmfyPc3Cz4rTWgZLI4iE/SX8Nt5l3o3oEjv3wwG5VD7xOjktzdwp5coac+kJV9l4jgg== + dependencies: + mdast-util-footnote "^0.1.0" + micromark-extension-footnote "^0.3.0" + +remark-gfm@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/remark-gfm/-/remark-gfm-1.0.0.tgz#9213643001be3f277da6256464d56fd28c3b3c0d" + integrity sha512-KfexHJCiqvrdBZVbQ6RopMZGwaXz6wFJEfByIuEwGf0arvITHjiKKZ1dpXujjH9KZdm1//XJQwgfnJ3lmXaDPA== + dependencies: + mdast-util-gfm "^0.1.0" + micromark-extension-gfm "^0.3.0" + +remark-parse@^9.0.0: + version "9.0.0" + resolved "https://registry.npmmirror.com/remark-parse/-/remark-parse-9.0.0.tgz#4d20a299665880e4f4af5d90b7c7b8a935853640" + integrity sha512-geKatMwSzEXKHuzBNU1z676sGcDcFoChMK38TgdHJNAYfFtsfHDQG7MoJAjs6sgYMqyLduCYWDIWZIxiPeafEw== + dependencies: + mdast-util-from-markdown "^0.8.0" + +remark-retext@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/remark-retext/-/remark-retext-4.0.0.tgz#255ed98ac3e0a68da5c6ba4f172299b8d062bb28" + integrity sha512-cYCchalpf25bTtfXF24ribYvqytPKq0TiEhqQDBHvVEEsApebwruPWP1cTcvTFBidmpXyqzycm+y8ng7Kmvc8Q== + dependencies: + mdast-util-to-nlcst "^4.0.0" + +remark-stringify@^9.0.0, remark-stringify@^9.0.1: + version "9.0.1" + resolved "https://registry.npmmirror.com/remark-stringify/-/remark-stringify-9.0.1.tgz#576d06e910548b0a7191a71f27b33f1218862894" + integrity sha512-mWmNg3ZtESvZS8fv5PTvaPckdL4iNlCHTt8/e/8oN08nArHRHjNZMKzA/YW3+p7/lYqIw4nx1XsjCBo/AxNChg== + dependencies: + mdast-util-to-markdown "^0.6.0" + +remark@^13.0.0: + version "13.0.0" + resolved "https://registry.npmmirror.com/remark/-/remark-13.0.0.tgz#d15d9bf71a402f40287ebe36067b66d54868e425" + integrity sha512-HDz1+IKGtOyWN+QgBiAT0kn+2s6ovOxHyPAFGKVE81VSzJ+mq7RwHFledEvB5F1p4iJvOah/LOKdFuzvRnNLCA== + dependencies: + remark-parse "^9.0.0" + remark-stringify "^9.0.0" + unified "^9.1.0" + +remove-trailing-separator@^1.0.1: + version "1.1.0" + resolved "https://registry.npmmirror.com/remove-trailing-separator/-/remove-trailing-separator-1.1.0.tgz#c24bce2a283adad5bc3f58e0d48249b92379d8ef" + integrity sha512-/hS+Y0u3aOfIETiaiirUFwDBDzmXPvO+jAfKTitUngIPzdKc6Z0LoFjM/CK5PL4C+eKwHohlHAb6H0VFfmmUsw== + +renderkid@^2.0.4: + version "2.0.7" + resolved "https://registry.npmmirror.com/renderkid/-/renderkid-2.0.7.tgz#464f276a6bdcee606f4a15993f9b29fc74ca8609" + integrity sha512-oCcFyxaMrKsKcTY59qnCAtmDVSLfPbrv6A3tVbPdFMMrv5jaK10V6m40cKsoPNhAqN6rmHW9sswW4o3ruSrwUQ== + dependencies: + css-select "^4.1.3" + dom-converter "^0.2.0" + htmlparser2 "^6.1.0" + lodash "^4.17.21" + strip-ansi "^3.0.1" + +repeat-string@^1.0.0: + version "1.6.1" + resolved "https://registry.npmmirror.com/repeat-string/-/repeat-string-1.6.1.tgz#8dcae470e1c88abc2d600fff4a776286da75e637" + integrity sha512-PV0dzCYDNfRi1jCDbJzpW7jNNDRuCOG/jI5ctQcGKt/clZD+YcPS3yIlWuTJMmESC8aevCFmWJy5wjAFgNqN6w== + +request@^2.88.0, request@^2.88.2: + version "2.88.2" + resolved "https://registry.npmmirror.com/request/-/request-2.88.2.tgz#d73c918731cb5a87da047e207234146f664d12b3" + integrity sha512-MsvtOrfG9ZcrOwAW+Qi+F6HbD0CWXEh9ou77uOb7FM2WPhwT7smM833PzanhJLsgXjN89Ir6V2PczXNnMpwKhw== + dependencies: + aws-sign2 "~0.7.0" + aws4 "^1.8.0" + caseless "~0.12.0" + combined-stream "~1.0.6" + extend "~3.0.2" + forever-agent "~0.6.1" + form-data "~2.3.2" + har-validator "~5.1.3" + http-signature "~1.2.0" + is-typedarray "~1.0.0" + isstream "~0.1.2" + json-stringify-safe "~5.0.1" + mime-types "~2.1.19" + oauth-sign "~0.9.0" + performance-now "^2.1.0" + qs "~6.5.2" + safe-buffer "^5.1.2" + tough-cookie "~2.5.0" + tunnel-agent "^0.6.0" + uuid "^3.3.2" + +require-directory@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" + integrity sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q== + +require-from-string@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/require-from-string/-/require-from-string-2.0.2.tgz#89a7fdd938261267318eafe14f9c32e598c36909" + integrity sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw== + +require-main-filename@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/require-main-filename/-/require-main-filename-2.0.0.tgz#d0b329ecc7cc0f61649f62215be69af54aa8989b" + integrity sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg== + +require-package-name@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/require-package-name/-/require-package-name-2.0.1.tgz#c11e97276b65b8e2923f75dabf5fb2ef0c3841b9" + integrity sha512-uuoJ1hU/k6M0779t3VMVIYpb2VMJk05cehCaABFhXaibcbvfgR8wKiozLjVFSzJPmQMRqIcO0HMyTFqfV09V6Q== + +requires-port@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/requires-port/-/requires-port-1.0.0.tgz#925d2601d39ac485e091cf0da5c6e694dc3dcaff" + integrity sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ== + +resize-observer-polyfill@^1.5.1: + version "1.5.1" + resolved "https://registry.npmmirror.com/resize-observer-polyfill/-/resize-observer-polyfill-1.5.1.tgz#0e9020dd3d21024458d4ebd27e23e40269810464" + integrity sha512-LwZrotdHOo12nQuZlHEmtuXdqGoOD0OhaxopaNFxWzInpEgaLWoVuAMbTzixuosCx2nEG58ngzW3vxdWoxIgdg== + +resolve-alpn@^1.0.0: + version "1.2.1" + resolved "https://registry.npmmirror.com/resolve-alpn/-/resolve-alpn-1.2.1.tgz#b7adbdac3546aaaec20b45e7d8265927072726f9" + integrity sha512-0a1F4l73/ZFZOakJnQ3FvkJ2+gSTQWz/r2KE5OdDY0TxPm5h4GkqkWWfM47T7HsbnOtcJVEF4epCVy6u7Q3K+g== + +resolve-cwd@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/resolve-cwd/-/resolve-cwd-3.0.0.tgz#0f0075f1bb2544766cf73ba6a6e2adfebcb13f2d" + integrity sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg== + dependencies: + resolve-from "^5.0.0" + +resolve-from@5.0.0, resolve-from@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/resolve-from/-/resolve-from-5.0.0.tgz#c35225843df8f776df21c57557bc087e9dfdfc69" + integrity sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw== + +resolve-from@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/resolve-from/-/resolve-from-4.0.0.tgz#4abcd852ad32dd7baabfe9b40e00a36db5f392e6" + integrity sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g== + +resolve-url-loader@^3.1.4: + version "3.1.4" + resolved "https://registry.npmmirror.com/resolve-url-loader/-/resolve-url-loader-3.1.4.tgz#3c16caebe0b9faea9c7cc252fa49d2353c412320" + integrity sha512-D3sQ04o0eeQEySLrcz4DsX3saHfsr8/N6tfhblxgZKXxMT2Louargg12oGNfoTRLV09GXhVUe5/qgA5vdgNigg== + dependencies: + adjust-sourcemap-loader "3.0.0" + camelcase "5.3.1" + compose-function "3.0.3" + convert-source-map "1.7.0" + es6-iterator "2.0.3" + loader-utils "1.2.3" + postcss "7.0.36" + rework "1.0.1" + rework-visit "1.0.0" + source-map "0.6.1" + +resolve-url@^0.2.1: + version "0.2.1" + resolved "https://registry.npmmirror.com/resolve-url/-/resolve-url-0.2.1.tgz#2c637fe77c893afd2a663fe21aa9080068e2052a" + integrity sha512-ZuF55hVUQaaczgOIwqWzkEcEidmlD/xl44x1UZnhOXcYuFN2S6+rcxpG+C1N3So0wvNI3DmJICUFfu2SxhBmvg== + +resolve@^1.1.7, resolve@^1.10.0, resolve@^1.14.2, resolve@^1.19.0, resolve@^1.20.0, resolve@^1.22.0: + version "1.22.0" + resolved "https://registry.npmmirror.com/resolve/-/resolve-1.22.0.tgz#5e0b8c67c15df57a89bdbabe603a002f21731198" + integrity sha512-Hhtrw0nLeSrFQ7phPp4OOcVjLPIeMnRlr5mcnVuMe7M/7eBn98A3hmFRLoFo3DLZkivSYwhRUJTyPyWAk56WLw== + dependencies: + is-core-module "^2.8.1" + path-parse "^1.0.7" + supports-preserve-symlinks-flag "^1.0.0" + +resolve@^2.0.0-next.3: + version "2.0.0-next.3" + resolved "https://registry.npmmirror.com/resolve/-/resolve-2.0.0-next.3.tgz#d41016293d4a8586a39ca5d9b5f15cbea1f55e46" + integrity sha512-W8LucSynKUIDu9ylraa7ueVZ7hc0uAgJBxVsQSKOXOyle8a93qXhcz+XAXZ8bIq2d6i4Ehddn6Evt+0/UwKk6Q== + dependencies: + is-core-module "^2.2.0" + path-parse "^1.0.6" + +responselike@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/responselike/-/responselike-1.0.2.tgz#918720ef3b631c5642be068f15ade5a46f4ba1e7" + integrity sha512-/Fpe5guzJk1gPqdJLJR5u7eG/gNY4nImjbRDaVWVMRhne55TCmj2i9Q+54PBRfatRC8v/rIiv9BN0pMd9OV5EQ== + dependencies: + lowercase-keys "^1.0.0" + +responselike@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/responselike/-/responselike-2.0.0.tgz#26391bcc3174f750f9a79eacc40a12a5c42d7723" + integrity sha512-xH48u3FTB9VsZw7R+vvgaKeLKzT6jOogbQhEe/jewwnZgzPcnyWui2Av6JpoYZF/91uueC+lqhWqeURw5/qhCw== + dependencies: + lowercase-keys "^2.0.0" + +restore-cursor@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/restore-cursor/-/restore-cursor-3.1.0.tgz#39f67c54b3a7a58cea5236d95cf0034239631f7e" + integrity sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA== + dependencies: + onetime "^5.1.0" + signal-exit "^3.0.2" + +retext-english@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/retext-english/-/retext-english-3.0.4.tgz#f978828d51fbcee842bc3807a45b7f709822ea8d" + integrity sha512-yr1PgaBDde+25aJXrnt3p1jvT8FVLVat2Bx8XeAWX13KXo8OT+3nWGU3HWxM4YFJvmfqvJYJZG2d7xxaO774gw== + dependencies: + parse-english "^4.0.0" + unherit "^1.0.4" + +retry@^0.12.0: + version "0.12.0" + resolved "https://registry.npmmirror.com/retry/-/retry-0.12.0.tgz#1b42a6266a21f07421d1b0b54b7dc167b01c013b" + integrity sha512-9LkiTwjUh6rT555DtE9rTX+BKByPfrMzEAtnlEtdEwr3Nkffwiihqe2bWADg+OQRjt9gl6ICdmB/ZFDCGAtSow== + +reusify@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/reusify/-/reusify-1.0.4.tgz#90da382b1e126efc02146e90845a88db12925d76" + integrity sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw== + +rework-visit@1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/rework-visit/-/rework-visit-1.0.0.tgz#9945b2803f219e2f7aca00adb8bc9f640f842c9a" + integrity sha512-W6V2fix7nCLUYX1v6eGPrBOZlc03/faqzP4sUxMAJMBMOPYhfV/RyLegTufn5gJKaOITyi+gvf0LXDZ9NzkHnQ== + +rework@1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/rework/-/rework-1.0.1.tgz#30806a841342b54510aa4110850cd48534144aa7" + integrity sha512-eEjL8FdkdsxApd0yWVZgBGzfCQiT8yqSc2H1p4jpZpQdtz7ohETiDMoje5PlM8I9WgkqkreVxFUKYOiJdVWDXw== + dependencies: + convert-source-map "^0.3.3" + css "^2.0.0" + +rimraf@^2.6.2: + version "2.7.1" + resolved "https://registry.npmmirror.com/rimraf/-/rimraf-2.7.1.tgz#35797f13a7fdadc566142c29d4f07ccad483e3ec" + integrity sha512-uWjbaKIK3T1OSVptzX7Nl6PvQ3qAGtKEtVRjRuazjfL3Bx5eI409VZSqgND+4UNnmzLVdPj9FqFJNPqBZFve4w== + dependencies: + glob "^7.1.3" + +rimraf@^3.0.0, rimraf@^3.0.2: + version "3.0.2" + resolved "https://registry.npmmirror.com/rimraf/-/rimraf-3.0.2.tgz#f1a5402ba6220ad52cc1282bac1ae3aa49fd061a" + integrity sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA== + dependencies: + glob "^7.1.3" + +run-async@^2.4.0: + version "2.4.1" + resolved "https://registry.npmmirror.com/run-async/-/run-async-2.4.1.tgz#8440eccf99ea3e70bd409d49aab88e10c189a455" + integrity sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ== + +run-parallel@^1.1.9: + version "1.2.0" + resolved "https://registry.npmmirror.com/run-parallel/-/run-parallel-1.2.0.tgz#66d1368da7bdf921eb9d95bd1a9229e7f21a43ee" + integrity sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA== + dependencies: + queue-microtask "^1.2.2" + +rxjs@^6.6.0: + version "6.6.7" + resolved "https://registry.npmmirror.com/rxjs/-/rxjs-6.6.7.tgz#90ac018acabf491bf65044235d5863c4dab804c9" + integrity sha512-hTdwr+7yYNIT5n4AMYp85KA6yw2Va0FLa3Rguvbpa4W3I5xynaBZo41cM3XM+4Q6fRMj3sBYIR1VAmZMXYJvRQ== + dependencies: + tslib "^1.9.0" + +safe-buffer@5.1.2, safe-buffer@~5.1.0, safe-buffer@~5.1.1: + version "5.1.2" + resolved "https://registry.npmmirror.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" + integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== + +safe-buffer@5.2.1, safe-buffer@^5.0.1, safe-buffer@^5.1.0, safe-buffer@^5.1.2, safe-buffer@~5.2.0: + version "5.2.1" + resolved "https://registry.npmmirror.com/safe-buffer/-/safe-buffer-5.2.1.tgz#1eaf9fa9bdb1fdd4ec75f58f9cdb4e6b7827eec6" + integrity sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ== + +"safer-buffer@>= 2.1.2 < 3", safer-buffer@^2.0.2, safer-buffer@^2.1.0, safer-buffer@~2.1.0: + version "2.1.2" + resolved "https://registry.npmmirror.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" + integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== + +sanitize-html@^1.27.5: + version "1.27.5" + resolved "https://registry.npmmirror.com/sanitize-html/-/sanitize-html-1.27.5.tgz#6c8149462adb23e360e1bb71cc0bae7f08c823c7" + integrity sha512-M4M5iXDAUEcZKLXkmk90zSYWEtk5NH3JmojQxKxV371fnMh+x9t1rqdmXaGoyEHw3z/X/8vnFhKjGL5xFGOJ3A== + dependencies: + htmlparser2 "^4.1.0" + lodash "^4.17.15" + parse-srcset "^1.0.2" + postcss "^7.0.27" + +sass-graph@2.2.5: + version "2.2.5" + resolved "https://registry.npmmirror.com/sass-graph/-/sass-graph-2.2.5.tgz#a981c87446b8319d96dce0671e487879bd24c2e8" + integrity sha512-VFWDAHOe6mRuT4mZRd4eKE+d8Uedrk6Xnh7Sh9b4NGufQLQjOrvf/MQoOdx+0s92L89FeyUUNfU597j/3uNpag== + dependencies: + glob "^7.0.0" + lodash "^4.0.0" + scss-tokenizer "^0.2.3" + yargs "^13.3.2" + +sass-loader@^10.1.1: + version "10.2.1" + resolved "https://registry.npmmirror.com/sass-loader/-/sass-loader-10.2.1.tgz#17e51df313f1a7a203889ce8ff91be362651276e" + integrity sha512-RRvWl+3K2LSMezIsd008ErK4rk6CulIMSwrcc2aZvjymUgKo/vjXGp1rSWmfTUX7bblEOz8tst4wBwWtCGBqKA== + dependencies: + klona "^2.0.4" + loader-utils "^2.0.0" + neo-async "^2.6.2" + schema-utils "^3.0.0" + semver "^7.3.2" + +sax@>=0.6.0, sax@^1.2.4, sax@~1.2.4: + version "1.2.4" + resolved "https://registry.npmmirror.com/sax/-/sax-1.2.4.tgz#2816234e2378bddc4e5354fab5caa895df7100d9" + integrity sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw== + +scheduler@^0.23.0: + version "0.23.0" + resolved "https://registry.npmmirror.com/scheduler/-/scheduler-0.23.0.tgz#ba8041afc3d30eb206a487b6b384002e4e61fdfe" + integrity sha512-CtuThmgHNg7zIZWAXi3AsyIzA3n4xx7aNyjwC2VJldO2LMVDhFK+63xGqq6CsJH4rTAt6/M+N4GhZiDYPx9eUw== + dependencies: + loose-envify "^1.1.0" + +schema-utils@2.7.0: + version "2.7.0" + resolved "https://registry.npmmirror.com/schema-utils/-/schema-utils-2.7.0.tgz#17151f76d8eae67fbbf77960c33c676ad9f4efc7" + integrity sha512-0ilKFI6QQF5nxDZLFn2dMjvc4hjg/Wkg7rHd3jK6/A4a1Hl9VFdQWvgB1UMGoU94pad1P/8N7fMcEnLnSiju8A== + dependencies: + "@types/json-schema" "^7.0.4" + ajv "^6.12.2" + ajv-keywords "^3.4.1" + +schema-utils@^2.6.5: + version "2.7.1" + resolved "https://registry.npmmirror.com/schema-utils/-/schema-utils-2.7.1.tgz#1ca4f32d1b24c590c203b8e7a50bf0ea4cd394d7" + integrity sha512-SHiNtMOUGWBQJwzISiVYKu82GiV4QYGePp3odlY1tuKO7gPtphAT5R/py0fA6xtbgLL/RvtJZnU9b8s0F1q0Xg== + dependencies: + "@types/json-schema" "^7.0.5" + ajv "^6.12.4" + ajv-keywords "^3.5.2" + +schema-utils@^3.0, schema-utils@^3.0.0, schema-utils@^3.1.0, schema-utils@^3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/schema-utils/-/schema-utils-3.1.1.tgz#bc74c4b6b6995c1d88f76a8b77bea7219e0c8281" + integrity sha512-Y5PQxS4ITlC+EahLuXaY86TXfR7Dc5lw294alXOq86JAHCihAIZfqv8nNCWvaEJvaC51uN9hbLGeV0cFBdH+Fw== + dependencies: + "@types/json-schema" "^7.0.8" + ajv "^6.12.5" + ajv-keywords "^3.5.2" + +scroll-into-view-if-needed@^2.2.25: + version "2.2.29" + resolved "https://registry.npmmirror.com/scroll-into-view-if-needed/-/scroll-into-view-if-needed-2.2.29.tgz#551791a84b7e2287706511f8c68161e4990ab885" + integrity sha512-hxpAR6AN+Gh53AdAimHM6C8oTN1ppwVZITihix+WqalywBeFcQ6LdQP5ABNl26nX8GTEL7VT+b8lKpdqq65wXg== + dependencies: + compute-scroll-into-view "^1.0.17" + +scss-tokenizer@^0.2.3: + version "0.2.3" + resolved "https://registry.npmmirror.com/scss-tokenizer/-/scss-tokenizer-0.2.3.tgz#8eb06db9a9723333824d3f5530641149847ce5d1" + integrity sha512-dYE8LhncfBUar6POCxMTm0Ln+erjeczqEvCJib5/7XNkdw1FkUGgwMPY360FY0FgPWQxHWCx29Jl3oejyGLM9Q== + dependencies: + js-base64 "^2.1.8" + source-map "^0.4.2" + +section-matter@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/section-matter/-/section-matter-1.0.0.tgz#e9041953506780ec01d59f292a19c7b850b84167" + integrity sha512-vfD3pmTzGpufjScBh50YHKzEu2lxBWhVEHsNGoEXmCmn2hKGfeNLYMzCJpe8cD7gqX7TJluOVpBkAequ6dgMmA== + dependencies: + extend-shallow "^2.0.1" + kind-of "^6.0.0" + +semver-diff@^3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/semver-diff/-/semver-diff-3.1.1.tgz#05f77ce59f325e00e2706afd67bb506ddb1ca32b" + integrity sha512-GX0Ix/CJcHyB8c4ykpHGIAvLyOwOobtM/8d+TQkAd81/bEjgPHrfba41Vpesr7jX/t8Uh+R3EX9eAS5be+jQYg== + dependencies: + semver "^6.3.0" + +"semver@2 || 3 || 4 || 5", semver@^5.5.0, semver@^5.7.1: + version "5.7.1" + resolved "https://registry.npmmirror.com/semver/-/semver-5.7.1.tgz#a954f931aeba508d307bbf069eff0c01c96116f7" + integrity sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ== + +semver@7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/semver/-/semver-7.0.0.tgz#5f3ca35761e47e05b206c6daff2cf814f0316b8e" + integrity sha512-+GB6zVA9LWh6zovYQLALHwv5rb2PHGlJi3lfiqIHxR0uuwCgefcOJc59v9fv1w8GbStwxuuqqAjI9NMAOOgq1A== + +semver@^6.0.0, semver@^6.1.1, semver@^6.1.2, semver@^6.2.0, semver@^6.3.0: + version "6.3.0" + resolved "https://registry.npmmirror.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d" + integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw== + +semver@^7.2.1, semver@^7.3.2, semver@^7.3.4, semver@^7.3.5, semver@^7.3.7: + version "7.3.7" + resolved "https://registry.npmmirror.com/semver/-/semver-7.3.7.tgz#12c5b649afdbf9049707796e22a4028814ce523f" + integrity sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g== + dependencies: + lru-cache "^6.0.0" + +send@0.18.0: + version "0.18.0" + resolved "https://registry.npmmirror.com/send/-/send-0.18.0.tgz#670167cc654b05f5aa4a767f9113bb371bc706be" + integrity sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg== + dependencies: + debug "2.6.9" + depd "2.0.0" + destroy "1.2.0" + encodeurl "~1.0.2" + escape-html "~1.0.3" + etag "~1.8.1" + fresh "0.5.2" + http-errors "2.0.0" + mime "1.6.0" + ms "2.1.3" + on-finished "2.4.1" + range-parser "~1.2.1" + statuses "2.0.1" + +sentence-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/sentence-case/-/sentence-case-3.0.4.tgz#3645a7b8c117c787fde8702056225bb62a45131f" + integrity sha512-8LS0JInaQMCRoQ7YUytAo/xUu5W2XnQxV2HI/6uM6U7CITS1RqPElr30V6uIqyMKM9lJGRVFy5/4CuzcixNYSg== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + upper-case-first "^2.0.2" + +serialize-javascript@^5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/serialize-javascript/-/serialize-javascript-5.0.1.tgz#7886ec848049a462467a97d3d918ebb2aaf934f4" + integrity sha512-SaaNal9imEO737H2c05Og0/8LUXG7EnsZyMa8MzkmuHoELfT6txuj0cMqRj6zfPKnmQ1yasR4PCJc8x+M4JSPA== + dependencies: + randombytes "^2.1.0" + +serialize-javascript@^6.0.0: + version "6.0.0" + resolved "https://registry.npmmirror.com/serialize-javascript/-/serialize-javascript-6.0.0.tgz#efae5d88f45d7924141da8b5c3a7a7e663fefeb8" + integrity sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag== + dependencies: + randombytes "^2.1.0" + +serve-static@1.15.0: + version "1.15.0" + resolved "https://registry.npmmirror.com/serve-static/-/serve-static-1.15.0.tgz#faaef08cffe0a1a62f60cad0c4e513cff0ac9540" + integrity sha512-XGuRDNjXUijsUL0vl6nSD7cwURuzEgglbOaFuZM9g3kwDXOWVTck0jLzjPzGD+TazWbboZYu52/9/XPdUgne9g== + dependencies: + encodeurl "~1.0.2" + escape-html "~1.0.3" + parseurl "~1.3.3" + send "0.18.0" + +set-blocking@^2.0.0, set-blocking@~2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/set-blocking/-/set-blocking-2.0.0.tgz#045f9782d011ae9a6803ddd382b24392b3d890f7" + integrity sha512-KiKBS8AnWGEyLzofFfmvKwpdPzqiy16LvQfK3yv/fVH7Bj13/wl3JSR1J+rfgRE9q7xUJK4qvgS8raSOeLUehw== + +setimmediate@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/setimmediate/-/setimmediate-1.0.5.tgz#290cbb232e306942d7d7ea9b83732ab7856f8285" + integrity sha512-MATJdZp8sLqDl/68LfQmbP8zKPLQNV6BIZoIgrscFDQ+RsvK/BxeDQOgyxKKoh0y/8h3BqVFnCqQ/gd+reiIXA== + +setprototypeof@1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/setprototypeof/-/setprototypeof-1.2.0.tgz#66c9a24a73f9fc28cbe66b09fed3d33dcaf1b424" + integrity sha512-E5LDX7Wrp85Kil5bhZv46j8jOeboKq5JMmYM3gVGdGH8xFpPWXUMsNrlODCrkoxMEeNi/XZIwuRvY4XNwYMJpw== + +shallow-clone@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/shallow-clone/-/shallow-clone-3.0.1.tgz#8f2981ad92531f55035b01fb230769a40e02efa3" + integrity sha512-/6KqX+GVUdqPuPPd2LxDDxzX6CAbjJehAAOKlNpqqUpAqPM6HeL8f+o3a+JsyGjn2lv0WY8UsTgUJjU9Ok55NA== + dependencies: + kind-of "^6.0.2" + +shallow-compare@^1.2.2: + version "1.2.2" + resolved "https://registry.npmmirror.com/shallow-compare/-/shallow-compare-1.2.2.tgz#fa4794627bf455a47c4f56881d8a6132d581ffdb" + integrity sha512-LUMFi+RppPlrHzbqmFnINTrazo0lPNwhcgzuAXVVcfy/mqPDrQmHAyz5bvV0gDAuRFrk804V0HpQ6u9sZ0tBeg== + +shallowequal@^1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/shallowequal/-/shallowequal-1.1.0.tgz#188d521de95b9087404fd4dcb68b13df0ae4e7f8" + integrity sha512-y0m1JoUZSlPAjXVtPPW70aZWfIL/dSP7AFkRnniLCrK/8MDKog3TySTBmckD+RObVxH0v4Tox67+F14PdED2oQ== + +sharp@^0.30.3: + version "0.30.6" + resolved "https://registry.npmmirror.com/sharp/-/sharp-0.30.6.tgz#02264e9826b5f1577509f70bb627716099778873" + integrity sha512-lSdVxFxcndzcXggDrak6ozdGJgmIgES9YVZWtAFrwi+a/H5vModaf51TghBtMPw+71sLxUsTy2j+aB7qLIODQg== + dependencies: + color "^4.2.3" + detect-libc "^2.0.1" + node-addon-api "^5.0.0" + prebuild-install "^7.1.0" + semver "^7.3.7" + simple-get "^4.0.1" + tar-fs "^2.1.1" + tunnel-agent "^0.6.0" + +shebang-command@^1.2.0: + version "1.2.0" + resolved "https://registry.npmmirror.com/shebang-command/-/shebang-command-1.2.0.tgz#44aac65b695b03398968c39f363fee5deafdf1ea" + integrity sha512-EV3L1+UQWGor21OmnvojK36mhg+TyIKDh3iFBKBohr5xeXIhNBcx8oWdgkTEEQ+BEFFYdLRuqMfd5L84N1V5Vg== + dependencies: + shebang-regex "^1.0.0" + +shebang-command@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/shebang-command/-/shebang-command-2.0.0.tgz#ccd0af4f8835fbdc265b82461aaf0c36663f34ea" + integrity sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA== + dependencies: + shebang-regex "^3.0.0" + +shebang-regex@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/shebang-regex/-/shebang-regex-1.0.0.tgz#da42f49740c0b42db2ca9728571cb190c98efea3" + integrity sha512-wpoSFAxys6b2a2wHZ1XpDSgD7N9iVjg29Ph9uV/uaP9Ex/KXlkTZTeddxDPSYQpgvzKLGJke2UU0AzoGCjNIvQ== + +shebang-regex@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/shebang-regex/-/shebang-regex-3.0.0.tgz#ae16f1644d873ecad843b0307b143362d4c42172" + integrity sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A== + +shell-quote@^1.7.3: + version "1.7.3" + resolved "https://registry.npmmirror.com/shell-quote/-/shell-quote-1.7.3.tgz#aa40edac170445b9a431e17bb62c0b881b9c4123" + integrity sha512-Vpfqwm4EnqGdlsBFNmHhxhElJYrdfcxPThu+ryKS5J8L/fhAwLazFZtq+S+TWZ9ANj2piSQLGj6NQg+lKPmxrw== + +side-channel@^1.0.4: + version "1.0.4" + resolved "https://registry.npmmirror.com/side-channel/-/side-channel-1.0.4.tgz#efce5c8fdc104ee751b25c58d4290011fa5ea2cf" + integrity sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw== + dependencies: + call-bind "^1.0.0" + get-intrinsic "^1.0.2" + object-inspect "^1.9.0" + +signal-exit@^3.0.0, signal-exit@^3.0.2, signal-exit@^3.0.3, signal-exit@^3.0.5, signal-exit@^3.0.6: + version "3.0.7" + resolved "https://registry.npmmirror.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9" + integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ== + +signedsource@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/signedsource/-/signedsource-1.0.0.tgz#1ddace4981798f93bd833973803d80d52e93ad6a" + integrity sha512-6+eerH9fEnNmi/hyM1DXcRK3pWdoMQtlkQ+ns0ntzunjKqp5i3sKCc80ym8Fib3iaYhdJUOPdhlJWj1tvge2Ww== + +simple-concat@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/simple-concat/-/simple-concat-1.0.1.tgz#f46976082ba35c2263f1c8ab5edfe26c41c9552f" + integrity sha512-cSFtAPtRhljv69IK0hTVZQ+OfE9nePi/rtJmw5UjHeVyVroEqJXP1sFztKUy1qU+xvz3u/sfYJLa947b7nAN2Q== + +simple-get@^4.0.0, simple-get@^4.0.1: + version "4.0.1" + resolved "https://registry.npmmirror.com/simple-get/-/simple-get-4.0.1.tgz#4a39db549287c979d352112fa03fd99fd6bc3543" + integrity sha512-brv7p5WgH0jmQJr1ZDDfKDOSeWWg+OVypG99A/5vYGPqJ6pxiaHLy8nxtFjBA7oMa01ebA9gfh1uMCFqOuXxvA== + dependencies: + decompress-response "^6.0.0" + once "^1.3.1" + simple-concat "^1.0.0" + +simple-swizzle@^0.2.2: + version "0.2.2" + resolved "https://registry.npmmirror.com/simple-swizzle/-/simple-swizzle-0.2.2.tgz#a4da6b635ffcccca33f70d17cb92592de95e557a" + integrity sha512-JA//kQgZtbuY83m+xT+tXJkmJncGMTFT+C+g2h2R9uxkYIrE2yy9sgmcLhCnw57/WSD+Eh3J97FPEDFnbXnDUg== + dependencies: + is-arrayish "^0.3.1" + +sisteransi@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/sisteransi/-/sisteransi-1.0.5.tgz#134d681297756437cc05ca01370d3a7a571075ed" + integrity sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg== + +slash@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/slash/-/slash-3.0.0.tgz#6539be870c165adbd5240220dbe361f1bc4d4634" + integrity sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q== + +slice-ansi@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/slice-ansi/-/slice-ansi-4.0.0.tgz#500e8dd0fd55b05815086255b3195adf2a45fe6b" + integrity sha512-qMCMfhY040cVHT43K9BFygqYbUPFZKHOg7K73mtTWJRb8pyP3fzf4Ixd5SzdEJQ6MRUg/WBnOLxghZtKKurENQ== + dependencies: + ansi-styles "^4.0.0" + astral-regex "^2.0.0" + is-fullwidth-code-point "^3.0.0" + +slugify@^1.6.1: + version "1.6.5" + resolved "https://registry.npmmirror.com/slugify/-/slugify-1.6.5.tgz#c8f5c072bf2135b80703589b39a3d41451fbe8c8" + integrity sha512-8mo9bslnBO3tr5PEVFzMPIWwWnipGS0xVbYf65zxDqfNwmzYn1LpiKNrR6DlClusuvo+hDHd1zKpmfAe83NQSQ== + +snake-case@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/snake-case/-/snake-case-3.0.4.tgz#4f2bbd568e9935abdfd593f34c691dadb49c452c" + integrity sha512-LAOh4z89bGQvl9pFfNF8V146i7o7/CqFPbqzYgP+yYzDIDeS9HaNFtXABamRW+AQzEVODcvE79ljJ+8a9YSdMg== + dependencies: + dot-case "^3.0.4" + tslib "^2.0.3" + +socket.io-adapter@~2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/socket.io-adapter/-/socket.io-adapter-2.1.0.tgz#edc5dc36602f2985918d631c1399215e97a1b527" + integrity sha512-+vDov/aTsLjViYTwS9fPy5pEtTkrbEKsw2M+oVSoFGw6OD1IpvlV1VPhUzNbofCQ8oyMbdYJqDtGdmHQK6TdPg== + +socket.io-client@3.1.3: + version "3.1.3" + resolved "https://registry.npmmirror.com/socket.io-client/-/socket.io-client-3.1.3.tgz#57ddcefea58cfab71f0e94c21124de8e3c5aa3e2" + integrity sha512-4sIGOGOmCg3AOgGi7EEr6ZkTZRkrXwub70bBB/F0JSkMOUFpA77WsL87o34DffQQ31PkbMUIadGOk+3tx1KGbw== + dependencies: + "@types/component-emitter" "^1.2.10" + backo2 "~1.0.2" + component-emitter "~1.3.0" + debug "~4.3.1" + engine.io-client "~4.1.0" + parseuri "0.0.6" + socket.io-parser "~4.0.4" + +socket.io-parser@~4.0.3, socket.io-parser@~4.0.4: + version "4.0.4" + resolved "https://registry.npmmirror.com/socket.io-parser/-/socket.io-parser-4.0.4.tgz#9ea21b0d61508d18196ef04a2c6b9ab630f4c2b0" + integrity sha512-t+b0SS+IxG7Rxzda2EVvyBZbvFPBCjJoyHuE0P//7OAsN23GItzDRdWa6ALxZI/8R5ygK7jAR6t028/z+7295g== + dependencies: + "@types/component-emitter" "^1.2.10" + component-emitter "~1.3.0" + debug "~4.3.1" + +socket.io@3.1.2: + version "3.1.2" + resolved "https://registry.npmmirror.com/socket.io/-/socket.io-3.1.2.tgz#06e27caa1c4fc9617547acfbb5da9bc1747da39a" + integrity sha512-JubKZnTQ4Z8G4IZWtaAZSiRP3I/inpy8c/Bsx2jrwGrTbKeVU5xd6qkKMHpChYeM3dWZSO0QACiGK+obhBNwYw== + dependencies: + "@types/cookie" "^0.4.0" + "@types/cors" "^2.8.8" + "@types/node" ">=10.0.0" + accepts "~1.3.4" + base64id "~2.0.0" + debug "~4.3.1" + engine.io "~4.1.0" + socket.io-adapter "~2.1.0" + socket.io-parser "~4.0.3" + +source-list-map@^2.0.0: + version "2.0.1" + resolved "https://registry.npmmirror.com/source-list-map/-/source-list-map-2.0.1.tgz#3993bd873bfc48479cca9ea3a547835c7c154b34" + integrity sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw== + +source-map-js@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" + integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== + +source-map-resolve@^0.5.2: + version "0.5.3" + resolved "https://registry.npmmirror.com/source-map-resolve/-/source-map-resolve-0.5.3.tgz#190866bece7553e1f8f267a2ee82c606b5509a1a" + integrity sha512-Htz+RnsXWk5+P2slx5Jh3Q66vhQj1Cllm0zvnaY98+NFx+Dv2CF/f5O/t8x+KaNdrdIAsruNzoh/KpialbqAnw== + dependencies: + atob "^2.1.2" + decode-uri-component "^0.2.0" + resolve-url "^0.2.1" + source-map-url "^0.4.0" + urix "^0.1.0" + +source-map-support@^0.5.17, source-map-support@^0.5.20, source-map-support@~0.5.20: + version "0.5.21" + resolved "https://registry.npmmirror.com/source-map-support/-/source-map-support-0.5.21.tgz#04fe7c7f9e1ed2d662233c28cb2b35b9f63f6e4f" + integrity sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w== + dependencies: + buffer-from "^1.0.0" + source-map "^0.6.0" + +source-map-url@^0.4.0: + version "0.4.1" + resolved "https://registry.npmmirror.com/source-map-url/-/source-map-url-0.4.1.tgz#0af66605a745a5a2f91cf1bbf8a7afbc283dec56" + integrity sha512-cPiFOTLUKvJFIg4SKVScy4ilPPW6rFgMgfuZJPNoDuMs3nC1HbMUycBoJw77xFIp6z1UJQJOfx6C9GMH80DiTw== + +source-map@0.6.1, source-map@^0.6.0, source-map@^0.6.1, source-map@~0.6.1: + version "0.6.1" + resolved "https://registry.npmmirror.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" + integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== + +source-map@^0.4.2: + version "0.4.4" + resolved "https://registry.npmmirror.com/source-map/-/source-map-0.4.4.tgz#eba4f5da9c0dc999de68032d8b4f76173652036b" + integrity sha512-Y8nIfcb1s/7DcobUz1yOO1GSp7gyL+D9zLHDehT7iRESqGSxjJ448Sg7rvfgsRJCnKLdSl11uGf0s9X80cH0/A== + dependencies: + amdefine ">=0.0.4" + +source-map@^0.7.3: + version "0.7.4" + resolved "https://registry.npmmirror.com/source-map/-/source-map-0.7.4.tgz#a9bbe705c9d8846f4e08ff6765acf0f1b0898656" + integrity sha512-l3BikUxvPOcn5E74dZiq5BGsTb5yEwhaTSzccU6t4sDOH8NWJCstKO5QT2CvtFoK6F0saL7p9xHAqHOlCPJygA== + +space-separated-tokens@^1.0.0: + version "1.1.5" + resolved "https://registry.npmmirror.com/space-separated-tokens/-/space-separated-tokens-1.1.5.tgz#85f32c3d10d9682007e917414ddc5c26d1aa6899" + integrity sha512-q/JSVd1Lptzhf5bkYm4ob4iWPjx0KiRe3sRFBNrVqbJkFaBm5vbbowy1mymoPNLRa52+oadOhJ+K49wsSeSjTA== + +spdx-correct@^3.0.0: + version "3.1.1" + resolved "https://registry.npmmirror.com/spdx-correct/-/spdx-correct-3.1.1.tgz#dece81ac9c1e6713e5f7d1b6f17d468fa53d89a9" + integrity sha512-cOYcUWwhCuHCXi49RhFRCyJEK3iPj1Ziz9DpViV3tbZOwXD49QzIN3MpOLJNxh2qwq2lJJZaKMVw9qNi4jTC0w== + dependencies: + spdx-expression-parse "^3.0.0" + spdx-license-ids "^3.0.0" + +spdx-exceptions@^2.1.0: + version "2.3.0" + resolved "https://registry.npmmirror.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz#3f28ce1a77a00372683eade4a433183527a2163d" + integrity sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A== + +spdx-expression-parse@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz#cf70f50482eefdc98e3ce0a6833e4a53ceeba679" + integrity sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q== + dependencies: + spdx-exceptions "^2.1.0" + spdx-license-ids "^3.0.0" + +spdx-license-ids@^3.0.0: + version "3.0.11" + resolved "https://registry.npmmirror.com/spdx-license-ids/-/spdx-license-ids-3.0.11.tgz#50c0d8c40a14ec1bf449bae69a0ea4685a9d9f95" + integrity sha512-Ctl2BrFiM0X3MANYgj3CkygxhRmr9mi6xhejbdO960nF6EDJApTYpn0BQnDKlnNBULKiCN1n3w9EBkHK8ZWg+g== + +split-on-first@^1.0.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/split-on-first/-/split-on-first-1.1.0.tgz#f610afeee3b12bce1d0c30425e76398b78249a5f" + integrity sha512-43ZssAJaMusuKWL8sKUBQXHWOpq8d6CfN/u1p4gUzfJkM05C8rxTmYrkIPTXapZpORA6LkkzcUulJ8FqA7Uudw== + +sponge-case@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/sponge-case/-/sponge-case-1.0.1.tgz#260833b86453883d974f84854cdb63aecc5aef4c" + integrity sha512-dblb9Et4DAtiZ5YSUZHLl4XhH4uK80GhAZrVXdN4O2P4gQ40Wa5UIOPUHlA/nFd2PLblBZWUioLMMAVrgpoYcA== + dependencies: + tslib "^2.0.3" + +sprintf-js@^1.1.1: + version "1.1.2" + resolved "https://registry.npmmirror.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.npmmirror.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c" + integrity sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g== + +sshpk@^1.7.0: + version "1.17.0" + resolved "https://registry.npmmirror.com/sshpk/-/sshpk-1.17.0.tgz#578082d92d4fe612b13007496e543fa0fbcbe4c5" + integrity sha512-/9HIEs1ZXGhSPE8X6Ccm7Nam1z8KcoCqPdI7ecm1N33EzAetWahvQWVqLZtaZQ+IDKX4IyA2o0gBzqIMkAagHQ== + dependencies: + asn1 "~0.2.3" + assert-plus "^1.0.0" + bcrypt-pbkdf "^1.0.0" + dashdash "^1.12.0" + ecc-jsbn "~0.1.1" + getpass "^0.1.1" + jsbn "~0.1.0" + safer-buffer "^2.0.2" + tweetnacl "~0.14.0" + +st@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/st/-/st-2.0.0.tgz#eabd11e7722863b8ee8cfbdd027cb25e76ff35e9" + integrity sha512-drN+aGYnrZPNYIymmNwIY7LXYJ8MqsqXj4fMRue3FOgGMdGjSX10fhJ3qx0sVQPhcWxhEaN4U/eWM4O4dbYNAw== + dependencies: + async-cache "^1.1.0" + bl "^4.0.0" + fd "~0.0.2" + mime "^2.4.4" + negotiator "~0.6.2" + optionalDependencies: + graceful-fs "^4.2.3" + +stable@^0.1.8: + version "0.1.8" + resolved "https://registry.npmmirror.com/stable/-/stable-0.1.8.tgz#836eb3c8382fe2936feaf544631017ce7d47a3cf" + integrity sha512-ji9qxRnOVfcuLDySj9qzhGSEFVobyt1kIOSkj1qZzYLzq7Tos/oUUWvotUPQLlrsidqsK6tBH89Bc9kL5zHA6w== + +stack-trace@^0.0.10: + version "0.0.10" + resolved "https://registry.npmmirror.com/stack-trace/-/stack-trace-0.0.10.tgz#547c70b347e8d32b4e108ea1a2a159e5fdde19c0" + integrity sha512-KGzahc7puUKkzyMt+IqAep+TVNbKP+k2Lmwhub39m1AsTSkaDutx56aDCo+HLDzf/D26BIHTJWNiTG1KAJiQCg== + +stackframe@^1.3.4: + version "1.3.4" + resolved "https://registry.npmmirror.com/stackframe/-/stackframe-1.3.4.tgz#b881a004c8c149a5e8efef37d51b16e412943310" + integrity sha512-oeVtt7eWQS+Na6F//S4kJ2K2VbRlS9D43mAlMyVpVWovy9o+jfgH8O9agzANzaiLjclA0oYzUXEM4PurhSUChw== + +statuses@2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/statuses/-/statuses-2.0.1.tgz#55cb000ccf1d48728bd23c685a063998cf1a1b63" + integrity sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ== + +"statuses@>= 1.5.0 < 2": + version "1.5.0" + resolved "https://registry.npmmirror.com/statuses/-/statuses-1.5.0.tgz#161c7dac177659fd9811f43771fa99381478628c" + integrity sha512-OpZ3zP+jT1PI7I8nemJX4AKmAX070ZkYPVWV/AaKTJl+tXCTGyVdC1a4SL8RUQYEwk/f34ZX8UTykN68FwrqAA== + +stdout-stream@^1.4.0: + version "1.4.1" + resolved "https://registry.npmmirror.com/stdout-stream/-/stdout-stream-1.4.1.tgz#5ac174cdd5cd726104aa0c0b2bd83815d8d535de" + integrity sha512-j4emi03KXqJWcIeF8eIXkjMFN1Cmb8gUlDYGeBALLPo5qdyTfA9bOtl8m33lRoC+vFMkP3gl0WsDr6+gzxbbTA== + dependencies: + readable-stream "^2.0.1" + +stream-parser@~0.3.1: + version "0.3.1" + resolved "https://registry.npmmirror.com/stream-parser/-/stream-parser-0.3.1.tgz#1618548694420021a1182ff0af1911c129761773" + integrity sha512-bJ/HgKq41nlKvlhccD5kaCr/P+Hu0wPNKPJOH7en+YrJu/9EgqUF+88w5Jb6KNcjOFMhfX4B2asfeAtIGuHObQ== + dependencies: + debug "2" + +streamsearch@0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/streamsearch/-/streamsearch-0.1.2.tgz#808b9d0e56fc273d809ba57338e929919a1a9f1a" + integrity sha512-jos8u++JKm0ARcSUTAZXOVC0mSox7Bhn6sBgty73P1f3JGf7yG2clTbBNHUdde/kdvP2FESam+vM6l8jBrNxHA== + +strict-uri-encode@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/strict-uri-encode/-/strict-uri-encode-2.0.0.tgz#b9c7330c7042862f6b142dc274bbcc5866ce3546" + integrity sha512-QwiXZgpRcKkhTj2Scnn++4PKtWsH0kpzZ62L2R6c/LUVYv7hVnZqcg2+sMuT6R7Jusu1vviK/MFsu6kNJfWlEQ== + +string-convert@^0.2.0: + version "0.2.1" + resolved "https://registry.npmmirror.com/string-convert/-/string-convert-0.2.1.tgz#6982cc3049fbb4cd85f8b24568b9d9bf39eeff97" + integrity sha512-u/1tdPl4yQnPBjnVrmdLo9gtuLvELKsAoRapekWggdiQNvvvum+jYF329d84NAa660KQw7pB2n36KrIKVoXa3A== + +string-env-interpolation@1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/string-env-interpolation/-/string-env-interpolation-1.0.1.tgz#ad4397ae4ac53fe6c91d1402ad6f6a52862c7152" + integrity sha512-78lwMoCcn0nNu8LszbP1UA7g55OeE4v7rCeWnM5B453rnNr4aq+5it3FEYtZrSEiMvHZOZ9Jlqb0OD0M2VInqg== + +string-natural-compare@^3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/string-natural-compare/-/string-natural-compare-3.0.1.tgz#7a42d58474454963759e8e8b7ae63d71c1e7fdf4" + integrity sha512-n3sPwynL1nwKi3WJ6AIsClwBMa0zTi54fn2oLU6ndfTSIO05xaznjSf15PcBZU6FNWbmN5Q6cxT4V5hGvB4taw== + +string-similarity@^1.2.2: + version "1.2.2" + resolved "https://registry.npmmirror.com/string-similarity/-/string-similarity-1.2.2.tgz#99b2c20a3c9bbb3903964eae1d89856db3d8db9b" + integrity sha512-IoHUjcw3Srl8nsPlW04U3qwWPk3oG2ffLM0tN853d/E/JlIvcmZmDY2Kz5HzKp4lEi2T7QD7Zuvjq/1rDw+XcQ== + dependencies: + lodash.every "^4.6.0" + lodash.flattendeep "^4.4.0" + lodash.foreach "^4.5.0" + lodash.map "^4.6.0" + lodash.maxby "^4.6.0" + +string-width@^1.0.1: + version "1.0.2" + resolved "https://registry.npmmirror.com/string-width/-/string-width-1.0.2.tgz#118bdf5b8cdc51a2a7e70d211e07e2b0b9b107d3" + integrity sha512-0XsVpQLnVCXHJfyEs8tC0zpTVIr5PKKsQtkT29IwupnPTjtPmQ3xT/4yCREF9hYkV/3M3kzcUTSAZT6a6h81tw== + dependencies: + code-point-at "^1.0.0" + is-fullwidth-code-point "^1.0.0" + strip-ansi "^3.0.0" + +"string-width@^1.0.2 || 2 || 3 || 4", string-width@^4.0.0, string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.2, string-width@^4.2.3: + version "4.2.3" + resolved "https://registry.npmmirror.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" + integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== + dependencies: + emoji-regex "^8.0.0" + is-fullwidth-code-point "^3.0.0" + strip-ansi "^6.0.1" + +string-width@^3.0.0, string-width@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/string-width/-/string-width-3.1.0.tgz#22767be21b62af1081574306f69ac51b62203961" + integrity sha512-vafcv6KjVZKSgz06oM/H6GDBrAtz8vdhQakGjFIvNrHA6y3HCF1CInLy+QLq8dTJPQ1b+KDUqDFctkdRW44e1w== + dependencies: + emoji-regex "^7.0.1" + is-fullwidth-code-point "^2.0.0" + strip-ansi "^5.1.0" + +string.prototype.matchall@^4.0.7: + version "4.0.7" + resolved "https://registry.npmmirror.com/string.prototype.matchall/-/string.prototype.matchall-4.0.7.tgz#8e6ecb0d8a1fb1fda470d81acecb2dba057a481d" + integrity sha512-f48okCX7JiwVi1NXCVWcFnZgADDC/n2vePlQ/KUCNqCikLLilQvwjMO8+BHVKvgzH0JB0J9LEPgxOGT02RoETg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + es-abstract "^1.19.1" + get-intrinsic "^1.1.1" + has-symbols "^1.0.3" + internal-slot "^1.0.3" + regexp.prototype.flags "^1.4.1" + side-channel "^1.0.4" + +string.prototype.trimend@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/string.prototype.trimend/-/string.prototype.trimend-1.0.5.tgz#914a65baaab25fbdd4ee291ca7dde57e869cb8d0" + integrity sha512-I7RGvmjV4pJ7O3kdf+LXFpVfdNOxtCW/2C8f6jNiW4+PQchwxkCDzlk1/7p+Wl4bqFIZeF47qAHXLuHHWKAxog== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.19.5" + +string.prototype.trimstart@^1.0.5: + version "1.0.5" + resolved "https://registry.npmmirror.com/string.prototype.trimstart/-/string.prototype.trimstart-1.0.5.tgz#5466d93ba58cfa2134839f81d7f42437e8c01fef" + integrity sha512-THx16TJCGlsN0o6dl2o6ncWUsdgnLRSA23rRE5pyGBw/mLr3Ej/R2LaqCtgP8VNMGZsvMWnf9ooZPyY2bHvUFg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.19.5" + +string_decoder@^1.1.1: + version "1.3.0" + resolved "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.3.0.tgz#42f114594a46cf1a8e30b0a84f56c78c3edac21e" + integrity sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA== + dependencies: + safe-buffer "~5.2.0" + +string_decoder@~0.10.x: + version "0.10.31" + resolved "https://registry.npmmirror.com/string_decoder/-/string_decoder-0.10.31.tgz#62e203bc41766c6c28c9fc84301dab1c5310fa94" + integrity sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ== + +string_decoder@~1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/string_decoder/-/string_decoder-1.1.1.tgz#9cf1611ba62685d7030ae9e4ba34149c3af03fc8" + integrity sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg== + dependencies: + safe-buffer "~5.1.0" + +stringify-entities@^3.0.1: + version "3.1.0" + resolved "https://registry.npmmirror.com/stringify-entities/-/stringify-entities-3.1.0.tgz#b8d3feac256d9ffcc9fa1fefdcf3ca70576ee903" + integrity sha512-3FP+jGMmMV/ffZs86MoghGqAoqXAdxLrJP4GUdrDN1aIScYih5tuIO3eF4To5AJZ79KDZ8Fpdy7QJnK8SsL1Vg== + dependencies: + character-entities-html4 "^1.0.0" + character-entities-legacy "^1.0.0" + xtend "^4.0.0" + +stringify-object@^3.3.0: + version "3.3.0" + resolved "https://registry.npmmirror.com/stringify-object/-/stringify-object-3.3.0.tgz#703065aefca19300d3ce88af4f5b3956d7556629" + integrity sha512-rHqiFh1elqCQ9WPLIC8I0Q/g/wj5J1eMkyoiD6eoQApWHP0FtlK7rqnhmabL5VUY9JQCcqwwvlOaSuutekgyrw== + dependencies: + get-own-enumerable-property-symbols "^3.0.0" + is-obj "^1.0.1" + is-regexp "^1.0.0" + +strip-ansi@^3.0.0, strip-ansi@^3.0.1: + version "3.0.1" + resolved "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-3.0.1.tgz#6a385fb8853d952d5ff05d0e8aaf94278dc63dcf" + integrity sha512-VhumSSbBqDTP8p2ZLKj40UjBCV4+v8bUSEpUb4KjRgWk9pbqGF4REFj6KEagidb2f/M6AzC0EmFyDNGaw9OCzg== + dependencies: + ansi-regex "^2.0.0" + +strip-ansi@^5.0.0, strip-ansi@^5.1.0, strip-ansi@^5.2.0: + version "5.2.0" + resolved "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-5.2.0.tgz#8c9a536feb6afc962bdfa5b104a5091c1ad9c0ae" + integrity sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA== + dependencies: + ansi-regex "^4.1.0" + +strip-ansi@^6.0.0, strip-ansi@^6.0.1: + version "6.0.1" + resolved "https://registry.npmmirror.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" + integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== + dependencies: + ansi-regex "^5.0.1" + +strip-bom-string@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/strip-bom-string/-/strip-bom-string-1.0.0.tgz#e5211e9224369fbb81d633a2f00044dc8cedad92" + integrity sha512-uCC2VHvQRYu+lMh4My/sFNmF2klFymLX1wHJeXnbEJERpV/ZsVuonzerjfrGpIGF7LBVa1O7i9kjiWvJiFck8g== + +strip-bom@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/strip-bom/-/strip-bom-3.0.0.tgz#2334c18e9c759f7bdd56fdef7e9ae3d588e68ed3" + integrity sha512-vavAMRXOgBVNF6nyEEmL3DBK19iRpDcoIwW+swQ+CbGiu7lju6t+JklA1MHweoWtadgt4ISVUsXLyDq34ddcwA== + +strip-comments@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/strip-comments/-/strip-comments-1.0.2.tgz#82b9c45e7f05873bee53f37168af930aa368679d" + integrity sha512-kL97alc47hoyIQSV165tTt9rG5dn4w1dNnBhOQ3bOU1Nc1hel09jnXANaHJ7vzHLd4Ju8kseDGzlev96pghLFw== + dependencies: + babel-extract-comments "^1.0.0" + babel-plugin-transform-object-rest-spread "^6.26.0" + +strip-eof@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/strip-eof/-/strip-eof-1.0.0.tgz#bb43ff5598a6eb05d89b59fcd129c983313606bf" + integrity sha512-7FCwGGmx8mD5xQd3RPUvnSpUXHM3BWuzjtpD4TXsfcZ9EL4azvVVUscFYwD9nx8Kh+uCBC00XBtAykoMHwTh8Q== + +strip-final-newline@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/strip-final-newline/-/strip-final-newline-2.0.0.tgz#89b852fb2fcbe936f6f4b3187afb0a12c1ab58ad" + integrity sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA== + +strip-indent@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/strip-indent/-/strip-indent-3.0.0.tgz#c32e1cee940b6b3432c771bc2c54bcce73cd3001" + integrity sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ== + dependencies: + min-indent "^1.0.0" + +strip-json-comments@^3.1.0, strip-json-comments@^3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/strip-json-comments/-/strip-json-comments-3.1.1.tgz#31f1281b3832630434831c310c01cccda8cbe006" + integrity sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig== + +strip-json-comments@~2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/strip-json-comments/-/strip-json-comments-2.0.1.tgz#3c531942e908c2697c0ec344858c286c7ca0a60a" + integrity sha512-4gB8na07fecVVkOI6Rs4e7T6NOTki5EmL7TUduTs6bu3EdnSycntVJ4re8kgZA+wx9IueI2Y11bfbgwtzuE0KQ== + +strip-outer@^1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/strip-outer/-/strip-outer-1.0.1.tgz#b2fd2abf6604b9d1e6013057195df836b8a9d631" + integrity sha512-k55yxKHwaXnpYGsOzg4Vl8+tDrWylxDEpknGjhTiZB8dFRU5rTo9CAzeycivxV3s+zlTKwrs6WxMxR95n26kwg== + dependencies: + escape-string-regexp "^1.0.2" + +strtok3@^6.2.4: + version "6.3.0" + resolved "https://registry.npmmirror.com/strtok3/-/strtok3-6.3.0.tgz#358b80ffe6d5d5620e19a073aa78ce947a90f9a0" + integrity sha512-fZtbhtvI9I48xDSywd/somNqgUHl2L2cstmXCCif0itOf96jeW18MBSyrLuNicYQVkvpOxkZtkzujiTJ9LW5Jw== + dependencies: + "@tokenizer/token" "^0.3.0" + peek-readable "^4.1.0" + +style-loader@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/style-loader/-/style-loader-2.0.0.tgz#9669602fd4690740eaaec137799a03addbbc393c" + integrity sha512-Z0gYUJmzZ6ZdRUqpg1r8GsaFKypE+3xAzuFeMuoHgjc9KZv3wMyCRjQIWEbhoFSq7+7yoHXySDJyyWQaPajeiQ== + dependencies: + loader-utils "^2.0.0" + schema-utils "^3.0.0" + +style-to-js@1.1.0: + version "1.1.0" + resolved "https://registry.npmmirror.com/style-to-js/-/style-to-js-1.1.0.tgz#631cbb20fce204019b3aa1fcb5b69d951ceac4ac" + integrity sha512-1OqefPDxGrlMwcbfpsTVRyzwdhr4W0uxYQzeA2F1CBc8WG04udg2+ybRnvh3XYL4TdHQrCahLtax2jc8xaE6rA== + dependencies: + style-to-object "0.3.0" + +style-to-object@0.3.0, style-to-object@^0.3.0: + version "0.3.0" + resolved "https://registry.npmmirror.com/style-to-object/-/style-to-object-0.3.0.tgz#b1b790d205991cc783801967214979ee19a76e46" + integrity sha512-CzFnRRXhzWIdItT3OmF8SQfWyahHhjq3HwcMNCNLn+N7klOOqPjMeG/4JSu77D7ypZdGvSzvkrbyeTMizz2VrA== + dependencies: + inline-style-parser "0.1.1" + +stylehacks@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/stylehacks/-/stylehacks-5.1.0.tgz#a40066490ca0caca04e96c6b02153ddc39913520" + integrity sha512-SzLmvHQTrIWfSgljkQCw2++C9+Ne91d/6Sp92I8c5uHTcy/PgeHamwITIbBW9wnFTY/3ZfSXR9HIL6Ikqmcu6Q== + dependencies: + browserslist "^4.16.6" + postcss-selector-parser "^6.0.4" + +stylis@4.0.13: + version "4.0.13" + resolved "https://registry.npmmirror.com/stylis/-/stylis-4.0.13.tgz#f5db332e376d13cc84ecfe5dace9a2a51d954c91" + integrity sha512-xGPXiFVl4YED9Jh7Euv2V220mriG9u4B2TA6Ybjc1catrstKD2PpIdU3U0RKpkVBC2EhmL/F0sPCr9vrFTNRag== + +subscriptions-transport-ws@^0.9.18: + version "0.9.19" + resolved "https://registry.npmmirror.com/subscriptions-transport-ws/-/subscriptions-transport-ws-0.9.19.tgz#10ca32f7e291d5ee8eb728b9c02e43c52606cdcf" + integrity sha512-dxdemxFFB0ppCLg10FTtRqH/31FNRL1y1BQv8209MK5I4CwALb7iihQg+7p65lFcIl8MHatINWBLOqpgU4Kyyw== + dependencies: + backo2 "^1.0.2" + eventemitter3 "^3.1.0" + iterall "^1.2.1" + symbol-observable "^1.0.4" + ws "^5.2.0 || ^6.0.0 || ^7.0.0" + +sudo-prompt@^8.2.0: + version "8.2.5" + resolved "https://registry.npmmirror.com/sudo-prompt/-/sudo-prompt-8.2.5.tgz#cc5ef3769a134bb94b24a631cc09628d4d53603e" + integrity sha512-rlBo3HU/1zAJUrkY6jNxDOC9eVYliG6nS4JA8u8KAshITd07tafMc/Br7xQwCSseXwJ2iCcHCE8SNWX3q8Z+kw== + +supports-color@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/supports-color/-/supports-color-2.0.0.tgz#535d045ce6b6363fa40117084629995e9df324c7" + integrity sha512-KKNVtd6pCYgPIKU4cp2733HWYCpplQhddZLBUryaAHou723x+FRzQ5Df824Fj+IyyuiQTRoub4SnIFfIcrp70g== + +supports-color@^5.3.0: + version "5.5.0" + resolved "https://registry.npmmirror.com/supports-color/-/supports-color-5.5.0.tgz#e2e69a44ac8772f78a1ec0b35b689df6530efc8f" + integrity sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow== + dependencies: + has-flag "^3.0.0" + +supports-color@^6.1.0: + version "6.1.0" + resolved "https://registry.npmmirror.com/supports-color/-/supports-color-6.1.0.tgz#0764abc69c63d5ac842dd4867e8d025e880df8f3" + integrity sha512-qe1jfm1Mg7Nq/NSh6XE24gPXROEVsWHxC1LIx//XNlD9iw7YZQGjZNjYN7xGaEG6iKdA8EtNFW6R0gjnVXp+wQ== + dependencies: + has-flag "^3.0.0" + +supports-color@^7.0.0, supports-color@^7.1.0: + version "7.2.0" + resolved "https://registry.npmmirror.com/supports-color/-/supports-color-7.2.0.tgz#1b7dcdcb32b8138801b3e478ba6a51caa89648da" + integrity sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw== + dependencies: + has-flag "^4.0.0" + +supports-color@^8.0.0: + version "8.1.1" + resolved "https://registry.npmmirror.com/supports-color/-/supports-color-8.1.1.tgz#cd6fc17e28500cff56c1b86c0a7fd4a54a73005c" + integrity sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q== + dependencies: + has-flag "^4.0.0" + +supports-preserve-symlinks-flag@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" + integrity sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w== + +svgo@1.3.2: + version "1.3.2" + resolved "https://registry.npmmirror.com/svgo/-/svgo-1.3.2.tgz#b6dc511c063346c9e415b81e43401145b96d4167" + integrity sha512-yhy/sQYxR5BkC98CY7o31VGsg014AKLEPxdfhora76l36hD9Rdy5NZA/Ocn6yayNPgSamYdtX2rFJdcv07AYVw== + dependencies: + chalk "^2.4.1" + coa "^2.0.2" + css-select "^2.0.0" + css-select-base-adapter "^0.1.1" + css-tree "1.0.0-alpha.37" + csso "^4.0.2" + js-yaml "^3.13.1" + mkdirp "~0.5.1" + object.values "^1.1.0" + sax "~1.2.4" + stable "^0.1.8" + unquote "~1.1.1" + util.promisify "~1.0.0" + +svgo@^2.7.0, svgo@^2.8.0: + version "2.8.0" + resolved "https://registry.npmmirror.com/svgo/-/svgo-2.8.0.tgz#4ff80cce6710dc2795f0c7c74101e6764cfccd24" + integrity sha512-+N/Q9kV1+F+UeWYoSiULYo4xYSDQlTgb+ayMobAXPwMnLvop7oxKMo9OzIrX5x3eS4L4f2UHhc9axXwY8DpChg== + dependencies: + "@trysound/sax" "0.2.0" + commander "^7.2.0" + css-select "^4.1.3" + css-tree "^1.1.3" + csso "^4.2.0" + picocolors "^1.0.0" + stable "^0.1.8" + +swap-case@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/swap-case/-/swap-case-2.0.2.tgz#671aedb3c9c137e2985ef51c51f9e98445bf70d9" + integrity sha512-kc6S2YS/2yXbtkSMunBtKdah4VFETZ8Oh6ONSmSd9bRxhqTrtARUCBUiWXH3xVPpvR7tz2CSnkuXVE42EcGnMw== + dependencies: + tslib "^2.0.3" + +symbol-observable@^1.0.4: + version "1.2.0" + resolved "https://registry.npmmirror.com/symbol-observable/-/symbol-observable-1.2.0.tgz#c22688aed4eab3cdc2dfeacbb561660560a00804" + integrity sha512-e900nM8RRtGhlV36KGEU9k65K3mPb1WV70OdjfxlG2EAuM1noi/E/BaW/uMhL7bPEssK8QV57vN3esixjUvcXQ== + +sync-fetch@0.3.0: + version "0.3.0" + resolved "https://registry.npmmirror.com/sync-fetch/-/sync-fetch-0.3.0.tgz#77246da949389310ad978ab26790bb05f88d1335" + integrity sha512-dJp4qg+x4JwSEW1HibAuMi0IIrBI3wuQr2GimmqB7OXR50wmwzfdusG+p39R9w3R6aFtZ2mzvxvWKQ3Bd/vx3g== + dependencies: + buffer "^5.7.0" + node-fetch "^2.6.1" + +table@^6.0.9: + version "6.8.0" + resolved "https://registry.npmmirror.com/table/-/table-6.8.0.tgz#87e28f14fa4321c3377ba286f07b79b281a3b3ca" + integrity sha512-s/fitrbVeEyHKFa7mFdkuQMWlH1Wgw/yEXMt5xACT4ZpzWFluehAxRtUUQKPuWhaLAWhFcVx6w3oC8VKaUfPGA== + dependencies: + ajv "^8.0.1" + lodash.truncate "^4.4.2" + slice-ansi "^4.0.0" + string-width "^4.2.3" + strip-ansi "^6.0.1" + +tailwindcss@^3.1.0: + version "3.1.3" + resolved "https://registry.npmmirror.com/tailwindcss/-/tailwindcss-3.1.3.tgz#b9ef2c1ae537c339679e8e89635af8e143d1c7eb" + integrity sha512-PRJNYdSIthrb8hjmAyymEyEN8Yo61TMXpzyFUpxULeeyRn3Y3gpvuw6FlRTKrJvK7thSGKRnhT36VovVx4WeMA== + dependencies: + arg "^5.0.2" + chokidar "^3.5.3" + color-name "^1.1.4" + detective "^5.2.1" + didyoumean "^1.2.2" + dlv "^1.1.3" + fast-glob "^3.2.11" + glob-parent "^6.0.2" + is-glob "^4.0.3" + lilconfig "^2.0.5" + normalize-path "^3.0.0" + object-hash "^3.0.0" + picocolors "^1.0.0" + postcss "^8.4.14" + postcss-import "^14.1.0" + postcss-js "^4.0.0" + postcss-load-config "^3.1.4" + postcss-nested "5.0.6" + postcss-selector-parser "^6.0.10" + postcss-value-parser "^4.2.0" + quick-lru "^5.1.1" + resolve "^1.22.0" + +tapable@^1.0.0: + version "1.1.3" + resolved "https://registry.npmmirror.com/tapable/-/tapable-1.1.3.tgz#a1fccc06b58db61fd7a45da2da44f5f3a3e67ba2" + integrity sha512-4WK/bYZmj8xLr+HUCODHGF1ZFzsYffasLUgEiMBY4fgtltdO6B4WJtlSbPaDTLpYTcGVwM2qLnFTICEcNxs3kA== + +tapable@^2.0, tapable@^2.1.1, tapable@^2.2.0: + version "2.2.1" + resolved "https://registry.npmmirror.com/tapable/-/tapable-2.2.1.tgz#1967a73ef4060a82f12ab96af86d52fdb76eeca0" + integrity sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ== + +tar-fs@^2.0.0, tar-fs@^2.1.1: + version "2.1.1" + resolved "https://registry.npmmirror.com/tar-fs/-/tar-fs-2.1.1.tgz#489a15ab85f1f0befabb370b7de4f9eb5cbe8784" + integrity sha512-V0r2Y9scmbDRLCNex/+hYzvp/zyYjvFbHPNgVTKfQvVrb6guiE/fxP+XblDNR011utopbkex2nM4dHNV6GDsng== + dependencies: + chownr "^1.1.1" + mkdirp-classic "^0.5.2" + pump "^3.0.0" + tar-stream "^2.1.4" + +tar-stream@^2.1.4: + version "2.2.0" + resolved "https://registry.npmmirror.com/tar-stream/-/tar-stream-2.2.0.tgz#acad84c284136b060dc3faa64474aa9aebd77287" + integrity sha512-ujeqbceABgwMZxEJnk2HDY2DlnUZ+9oEcb1KzTVfYHio0UE6dG71n60d8D2I4qNvleWrrXpmjpt7vZeF1LnMZQ== + dependencies: + bl "^4.0.3" + end-of-stream "^1.4.1" + fs-constants "^1.0.0" + inherits "^2.0.3" + readable-stream "^3.1.1" + +tar@^6.0.2: + version "6.1.11" + resolved "https://registry.npmmirror.com/tar/-/tar-6.1.11.tgz#6760a38f003afa1b2ffd0ffe9e9abbd0eab3d621" + integrity sha512-an/KZQzQUkZCkuoAA64hM92X0Urb6VpRhAFllDzz44U2mcD5scmT3zBc4VgVpkugF580+DQn8eAFSyoQt0tznA== + dependencies: + chownr "^2.0.0" + fs-minipass "^2.0.0" + minipass "^3.0.0" + minizlib "^2.1.1" + mkdirp "^1.0.3" + yallist "^4.0.0" + +term-size@^2.1.0: + version "2.2.1" + resolved "https://registry.npmmirror.com/term-size/-/term-size-2.2.1.tgz#2a6a54840432c2fb6320fea0f415531e90189f54" + integrity sha512-wK0Ri4fOGjv/XPy8SBHZChl8CM7uMc5VML7SqiQ0zG7+J5Vr+RMQDoHa2CNT6KHUnTGIXH34UDMkPzAUyapBZg== + +terser-webpack-plugin@^5.1.3, terser-webpack-plugin@^5.2.4: + version "5.3.3" + resolved "https://registry.npmmirror.com/terser-webpack-plugin/-/terser-webpack-plugin-5.3.3.tgz#8033db876dd5875487213e87c627bca323e5ed90" + integrity sha512-Fx60G5HNYknNTNQnzQ1VePRuu89ZVYWfjRAeT5rITuCY/1b08s49e5kSQwHDirKZWuoKOBRFS98EUUoZ9kLEwQ== + dependencies: + "@jridgewell/trace-mapping" "^0.3.7" + jest-worker "^27.4.5" + schema-utils "^3.1.1" + serialize-javascript "^6.0.0" + terser "^5.7.2" + +terser@^5.2.0, terser@^5.7.2: + version "5.14.1" + resolved "https://registry.npmmirror.com/terser/-/terser-5.14.1.tgz#7c95eec36436cb11cf1902cc79ac564741d19eca" + integrity sha512-+ahUAE+iheqBTDxXhTisdA8hgvbEG1hHOQ9xmNjeUJSoi6DU/gMrKNcfZjHkyY6Alnuyc+ikYJaxxfHkT3+WuQ== + dependencies: + "@jridgewell/source-map" "^0.3.2" + acorn "^8.5.0" + commander "^2.20.0" + source-map-support "~0.5.20" + +text-table@^0.2.0: + version "0.2.0" + resolved "https://registry.npmmirror.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4" + integrity sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw== + +through2@~0.4.1: + version "0.4.2" + resolved "https://registry.npmmirror.com/through2/-/through2-0.4.2.tgz#dbf5866031151ec8352bb6c4db64a2292a840b9b" + integrity sha512-45Llu+EwHKtAZYTPPVn3XZHBgakWMN3rokhEv5hu596XP+cNgplMg+Gj+1nmAvj+L0K7+N49zBKx5rah5u0QIQ== + dependencies: + readable-stream "~1.0.17" + xtend "~2.1.1" + +through@^2.3.6, through@^2.3.8: + version "2.3.8" + resolved "https://registry.npmmirror.com/through/-/through-2.3.8.tgz#0dd4c9ffaabc357960b1b724115d7e0e86a2e1f5" + integrity sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg== + +timers-ext@^0.1.7: + version "0.1.7" + resolved "https://registry.npmmirror.com/timers-ext/-/timers-ext-0.1.7.tgz#6f57ad8578e07a3fb9f91d9387d65647555e25c6" + integrity sha512-b85NUNzTSdodShTIbky6ZF02e8STtVVfD+fu4aXXShEELpozH+bCpJLYMPZbsABN2wDH7fJpqIoXxJpzbf0NqQ== + dependencies: + es5-ext "~0.10.46" + next-tick "1" + +timm@^1.6.1: + version "1.7.1" + resolved "https://registry.npmmirror.com/timm/-/timm-1.7.1.tgz#96bab60c7d45b5a10a8a4d0f0117c6b7e5aff76f" + integrity sha512-IjZc9KIotudix8bMaBW6QvMuq64BrJWFs1+4V0lXwWGQZwH+LnX87doAYhem4caOEusRP9/g6jVDQmZ8XOk1nw== + +tinycolor2@^1.4.1: + version "1.4.2" + resolved "https://registry.npmmirror.com/tinycolor2/-/tinycolor2-1.4.2.tgz#3f6a4d1071ad07676d7fa472e1fac40a719d8803" + integrity sha512-vJhccZPs965sV/L2sU4oRQVAos0pQXwsvTLkWYdqJ+a8Q5kPFzJTuOFwy7UniPli44NKQGAglksjvOcpo95aZA== + +title-case@^3.0.3: + version "3.0.3" + resolved "https://registry.npmmirror.com/title-case/-/title-case-3.0.3.tgz#bc689b46f02e411f1d1e1d081f7c3deca0489982" + integrity sha512-e1zGYRvbffpcHIrnuqT0Dh+gEJtDaxDSoG4JAIpq4oDFyooziLBIiYQv0GBT4FUAnUop5uZ1hiIAj7oAF6sOCA== + dependencies: + tslib "^2.0.3" + +tmp@^0.0.33: + version "0.0.33" + resolved "https://registry.npmmirror.com/tmp/-/tmp-0.0.33.tgz#6d34335889768d21b2bcda0aa277ced3b1bfadf9" + integrity sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw== + dependencies: + os-tmpdir "~1.0.2" + +tmp@^0.2.1: + version "0.2.1" + resolved "https://registry.npmmirror.com/tmp/-/tmp-0.2.1.tgz#8457fc3037dcf4719c251367a1af6500ee1ccf14" + integrity sha512-76SUhtfqR2Ijn+xllcI5P1oyannHNHByD80W1q447gU3mp9G9PSpGdWmjUOHRDPiHYacIk66W7ubDTuPF3BEtQ== + dependencies: + rimraf "^3.0.0" + +to-fast-properties@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e" + integrity sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog== + +to-readable-stream@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/to-readable-stream/-/to-readable-stream-1.0.0.tgz#ce0aa0c2f3df6adf852efb404a783e77c0475771" + integrity sha512-Iq25XBt6zD5npPhlLVXGFN3/gyR2/qODcKNNyTMd4vbm39HUaOiAM4PMq0eMVC/Tkxz+Zjdsc55g9yyz+Yq00Q== + +to-regex-range@^5.0.1: + version "5.0.1" + resolved "https://registry.npmmirror.com/to-regex-range/-/to-regex-range-5.0.1.tgz#1648c44aae7c8d988a326018ed72f5b4dd0392e4" + integrity sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ== + dependencies: + is-number "^7.0.0" + +toggle-selection@^1.0.6: + version "1.0.6" + resolved "https://registry.npmmirror.com/toggle-selection/-/toggle-selection-1.0.6.tgz#6e45b1263f2017fa0acc7d89d78b15b8bf77da32" + integrity sha512-BiZS+C1OS8g/q2RRbJmy59xpyghNBqrr6k5L/uKBGRsTfxmu3ffiRnd8mlGPUVayg8pvfi5urfnu8TU7DVOkLQ== + +toidentifier@1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/toidentifier/-/toidentifier-1.0.0.tgz#7e1be3470f1e77948bc43d94a3c8f4d7752ba553" + integrity sha512-yaOH/Pk/VEhBWWTlhI+qXxDFXlejDGcQipMlyxda9nthulaxLZUNcUqFxokp0vcYnvteJln5FNQDRrxj3YcbVw== + +toidentifier@1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/toidentifier/-/toidentifier-1.0.1.tgz#3be34321a88a820ed1bd80dfaa33e479fbb8dd35" + integrity sha512-o5sSPKEkg/DIQNmH43V0/uerLrpzVedkUh8tGNvaeXpfpuwjKenlSox/2O/BTlZUtEe+JG7s5YhEz608PlAHRA== + +token-types@^4.1.1: + version "4.2.0" + resolved "https://registry.npmmirror.com/token-types/-/token-types-4.2.0.tgz#b66bc3d67420c6873222a424eee64a744f4c2f13" + integrity sha512-P0rrp4wUpefLncNamWIef62J0v0kQR/GfDVji9WKY7GDCWy5YbVSrKUTam07iWPZQGy0zWNOfstYTykMmPNR7w== + dependencies: + "@tokenizer/token" "^0.3.0" + ieee754 "^1.2.1" + +tough-cookie@~2.5.0: + version "2.5.0" + resolved "https://registry.npmmirror.com/tough-cookie/-/tough-cookie-2.5.0.tgz#cd9fb2a0aa1d5a12b473bd9fb96fa3dcff65ade2" + integrity sha512-nlLsUzgm1kfLXSXfRZMc1KLAugd4hqJHDTvc2hDIwS3mZAfMEuMbc03SujMF+GEcpaX/qboeycw6iO8JwVv2+g== + dependencies: + psl "^1.1.28" + punycode "^2.1.1" + +tr46@~0.0.3: + version "0.0.3" + resolved "https://registry.npmmirror.com/tr46/-/tr46-0.0.3.tgz#8184fd347dac9cdc185992f3a6622e14b9d9ab6a" + integrity sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw== + +trim-newlines@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/trim-newlines/-/trim-newlines-3.0.1.tgz#260a5d962d8b752425b32f3a7db0dcacd176c144" + integrity sha512-c1PTsA3tYrIsLGkJkzHF+w9F2EyxfXGo4UyJc4pFL++FMjnq0HJS69T3M7d//gKrFKwy429bouPescbjecU+Zw== + +trim-repeated@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/trim-repeated/-/trim-repeated-1.0.0.tgz#e3646a2ea4e891312bf7eace6cfb05380bc01c21" + integrity sha512-pkonvlKk8/ZuR0D5tLW8ljt5I8kmxp2XKymhepUeOdCEfKpZaktSArkLHZt76OB1ZvO9bssUsDty4SWhLvZpLg== + dependencies: + escape-string-regexp "^1.0.2" + +trough@^1.0.0: + version "1.0.5" + resolved "https://registry.npmmirror.com/trough/-/trough-1.0.5.tgz#b8b639cefad7d0bb2abd37d433ff8293efa5f406" + integrity sha512-rvuRbTarPXmMb79SmzEp8aqXNKcK+y0XaB298IXueQ8I2PsrATcPBCSPyK/dDNa2iWOhKlfNnOjdAOTBU/nkFA== + +"true-case-path@^1.0.2": + version "1.0.3" + resolved "https://registry.npmmirror.com/true-case-path/-/true-case-path-1.0.3.tgz#f813b5a8c86b40da59606722b144e3225799f47d" + integrity sha512-m6s2OdQe5wgpFMC+pAJ+q9djG82O2jcHPOI6RNg1yy9rCYR+WD6Nbpl32fDpfC56nirdRy+opFa/Vk7HYhqaew== + dependencies: + glob "^7.1.2" + +"true-case-path@^2.2.1": + version "2.2.1" + resolved "https://registry.npmmirror.com/true-case-path/-/true-case-path-2.2.1.tgz#c5bf04a5bbec3fd118be4084461b3a27c4d796bf" + integrity sha512-0z3j8R7MCjy10kc/g+qg7Ln3alJTodw9aDuVWZa3uiWqfuBMKeAeP2ocWcxoyM3D73yz3Jt/Pu4qPr4wHSdB/Q== + +ts-node@^9: + version "9.1.1" + resolved "https://registry.npmmirror.com/ts-node/-/ts-node-9.1.1.tgz#51a9a450a3e959401bda5f004a72d54b936d376d" + integrity sha512-hPlt7ZACERQGf03M253ytLY3dHbGNGrAq9qIHWUY9XHYl1z7wYngSr3OQ5xmui8o2AaxsONxIzjafLUiWBo1Fg== + dependencies: + arg "^4.1.0" + create-require "^1.1.0" + diff "^4.0.1" + make-error "^1.1.1" + source-map-support "^0.5.17" + yn "3.1.1" + +tsconfig-paths@^3.14.1: + version "3.14.1" + resolved "https://registry.npmmirror.com/tsconfig-paths/-/tsconfig-paths-3.14.1.tgz#ba0734599e8ea36c862798e920bcf163277b137a" + integrity sha512-fxDhWnFSLt3VuTwtvJt5fpwxBHg5AdKWMsgcPOOIilyjymcYVZoCQF8fvFRezCNfblEXmi+PcM1eYHeOAgXCOQ== + dependencies: + "@types/json5" "^0.0.29" + json5 "^1.0.1" + minimist "^1.2.6" + strip-bom "^3.0.0" + +tslib@^1.10.0, tslib@^1.8.1, tslib@^1.9.0: + version "1.14.1" + resolved "https://registry.npmmirror.com/tslib/-/tslib-1.14.1.tgz#cf2d38bdc34a134bcaf1091c41f6619e2f672d00" + integrity sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg== + +tslib@^2, tslib@^2.0.3, tslib@^2.1.0, tslib@^2.4.0, tslib@~2.4.0: + version "2.4.0" + resolved "https://registry.npmmirror.com/tslib/-/tslib-2.4.0.tgz#7cecaa7f073ce680a05847aa77be941098f36dc3" + integrity sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ== + +tslib@~2.0.1: + version "2.0.3" + resolved "https://registry.npmmirror.com/tslib/-/tslib-2.0.3.tgz#8e0741ac45fc0c226e58a17bfc3e64b9bc6ca61c" + integrity sha512-uZtkfKblCEQtZKBF6EBXVZeQNl82yqtDQdv+eck8u7tdPxjLu2/lp5/uPW+um2tpuxINHWy3GhiccY7QgEaVHQ== + +tslib@~2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/tslib/-/tslib-2.1.0.tgz#da60860f1c2ecaa5703ab7d39bc05b6bf988b97a" + integrity sha512-hcVC3wYEziELGGmEEXue7D75zbwIIVUMWAVbHItGPx0ziyXxrOMQx4rQEVEV45Ut/1IotuEvwqPopzIOkDMf0A== + +tslib@~2.2.0: + version "2.2.0" + resolved "https://registry.npmmirror.com/tslib/-/tslib-2.2.0.tgz#fb2c475977e35e241311ede2693cee1ec6698f5c" + integrity sha512-gS9GVHRU+RGn5KQM2rllAlR3dU6m7AcpJKdtH8gFvQiC4Otgk98XnmMU+nZenHt/+VhnBPWwgrJsyrdcw6i23w== + +tslib@~2.3.0: + version "2.3.1" + resolved "https://registry.npmmirror.com/tslib/-/tslib-2.3.1.tgz#e8a335add5ceae51aa261d32a490158ef042ef01" + integrity sha512-77EbyPPpMz+FRFRuAFlWMtmgUWGe9UOG2Z25NqCwiIjRhOf5iKGuzSe5P2w1laq+FkRy4p+PCuVkJSGkzTEKVw== + +tsutils@^3.21.0: + version "3.21.0" + resolved "https://registry.npmmirror.com/tsutils/-/tsutils-3.21.0.tgz#b48717d394cea6c1e096983eed58e9d61715b623" + integrity sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA== + dependencies: + tslib "^1.8.1" + +tunnel-agent@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/tunnel-agent/-/tunnel-agent-0.6.0.tgz#27a5dea06b36b04a0a9966774b290868f0fc40fd" + integrity sha512-McnNiV1l8RYeY8tBgEpuodCC1mLUdbSN+CYBL7kJsJNInOP8UjDDEwdk6Mw60vdLLrr5NHKZhMAOSrR2NZuQ+w== + dependencies: + safe-buffer "^5.0.1" + +tweetnacl@^0.14.3, tweetnacl@~0.14.0: + version "0.14.5" + resolved "https://registry.npmmirror.com/tweetnacl/-/tweetnacl-0.14.5.tgz#5ae68177f192d4456269d108afa93ff8743f4f64" + integrity sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA== + +type-check@^0.4.0, type-check@~0.4.0: + version "0.4.0" + resolved "https://registry.npmmirror.com/type-check/-/type-check-0.4.0.tgz#07b8203bfa7056c0657050e3ccd2c37730bab8f1" + integrity sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew== + dependencies: + prelude-ls "^1.2.1" + +type-fest@^0.18.0: + version "0.18.1" + resolved "https://registry.npmmirror.com/type-fest/-/type-fest-0.18.1.tgz#db4bc151a4a2cf4eebf9add5db75508db6cc841f" + integrity sha512-OIAYXk8+ISY+qTOwkHtKqzAuxchoMiD9Udx+FSGQDuiRR+PJKJHc2NJAXlbhkGwTt/4/nKZxELY1w3ReWOL8mw== + +type-fest@^0.20.2: + version "0.20.2" + resolved "https://registry.npmmirror.com/type-fest/-/type-fest-0.20.2.tgz#1bf207f4b28f91583666cb5fbd327887301cd5f4" + integrity sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ== + +type-fest@^0.21.3: + version "0.21.3" + resolved "https://registry.npmmirror.com/type-fest/-/type-fest-0.21.3.tgz#d260a24b0198436e133fa26a524a6d65fa3b2e37" + integrity sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w== + +type-fest@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/type-fest/-/type-fest-0.6.0.tgz#8d2a2370d3df886eb5c90ada1c5bf6188acf838b" + integrity sha512-q+MB8nYR1KDLrgr4G5yemftpMC7/QLqVndBmEEdqzmNj5dcFOO4Oo8qlwZE3ULT3+Zim1F8Kq4cBnikNhlCMlg== + +type-fest@^0.8.0, type-fest@^0.8.1: + version "0.8.1" + resolved "https://registry.npmmirror.com/type-fest/-/type-fest-0.8.1.tgz#09e249ebde851d3b1e48d27c105444667f17b83d" + integrity sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA== + +type-is@^1.6.4, type-is@~1.6.18: + version "1.6.18" + resolved "https://registry.npmmirror.com/type-is/-/type-is-1.6.18.tgz#4e552cd05df09467dcbc4ef739de89f2cf37c131" + integrity sha512-TkRKr9sUTxEH8MdfuCSP7VizJyzRNMjj2J2do2Jr3Kym598JVdEksuzPQCnlFPW4ky9Q+iA+ma9BGm06XQBy8g== + dependencies: + media-typer "0.3.0" + mime-types "~2.1.24" + +type-of@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/type-of/-/type-of-2.0.1.tgz#e72a1741896568e9f628378d816d6912f7f23972" + integrity sha512-39wxbwHdQ2sTiBB8wAzKfQ9GN+om8w+sjNWzr+vZJR5AMD5J+J7Yc8AtXnU9r/r2c8XiDZ/smxutDmZehX/qpQ== + +type@^1.0.1: + version "1.2.0" + resolved "https://registry.npmmirror.com/type/-/type-1.2.0.tgz#848dd7698dafa3e54a6c479e759c4bc3f18847a0" + integrity sha512-+5nt5AAniqsCnu2cEQQdpzCAh33kVx8n0VoFidKpB1dVVLAN/F+bgVOqOJqOnEnrhp222clB5p3vUlD+1QAnfg== + +type@^2.5.0: + version "2.6.0" + resolved "https://registry.npmmirror.com/type/-/type-2.6.0.tgz#3ca6099af5981d36ca86b78442973694278a219f" + integrity sha512-eiDBDOmkih5pMbo9OqsqPRGMljLodLcwd5XD5JbtNB0o89xZAwynY9EdCDsJU7LtcVCClu9DvM7/0Ep1hYX3EQ== + +typedarray-to-buffer@^3.1.5: + version "3.1.5" + resolved "https://registry.npmmirror.com/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz#a97ee7a9ff42691b9f783ff1bc5112fe3fca9080" + integrity sha512-zdu8XMNEDepKKR+XYOXAVPtWui0ly0NtohUscw+UmaHiAWT8hrV1rr//H6V+0DvJ3OQ19S979M0laLfX8rm82Q== + dependencies: + is-typedarray "^1.0.0" + +typedarray@^0.0.6: + version "0.0.6" + resolved "https://registry.npmmirror.com/typedarray/-/typedarray-0.0.6.tgz#867ac74e3864187b1d3d47d996a78ec5c8830777" + integrity sha512-/aCDEGatGvZ2BIk+HmLf4ifCJFwvKFNb9/JeZPMulfgFracn9QFcAf5GO8B/mweUjSoblS5In0cWhqpfs/5PQA== + +ua-parser-js@^0.7.30: + version "0.7.31" + resolved "https://registry.npmmirror.com/ua-parser-js/-/ua-parser-js-0.7.31.tgz#649a656b191dffab4f21d5e053e27ca17cbff5c6" + integrity sha512-qLK/Xe9E2uzmYI3qLeOmI0tEOt+TBBQyUIAh4aAgU05FVYzeZrKUdkAZfBNVGRaHVgV0TDkdEngJSw/SyQchkQ== + +unbox-primitive@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/unbox-primitive/-/unbox-primitive-1.0.2.tgz#29032021057d5e6cdbd08c5129c226dff8ed6f9e" + integrity sha512-61pPlCD9h51VoreyJ0BReideM3MDKMKnh6+V9L08331ipq6Q8OFXZYiqP6n/tbHx4s5I9uRhcye6BrbkizkBDw== + dependencies: + call-bind "^1.0.2" + has-bigints "^1.0.2" + has-symbols "^1.0.3" + which-boxed-primitive "^1.0.2" + +unc-path-regex@^0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/unc-path-regex/-/unc-path-regex-0.1.2.tgz#e73dd3d7b0d7c5ed86fbac6b0ae7d8c6a69d50fa" + integrity sha512-eXL4nmJT7oCpkZsHZUOJo8hcX3GbsiDOa0Qu9F646fi8dT3XuSVopVqAcEiVzSKKH7UoDti23wNX3qGFxcW5Qg== + +underscore.string@^3.3.6: + version "3.3.6" + resolved "https://registry.npmmirror.com/underscore.string/-/underscore.string-3.3.6.tgz#ad8cf23d7423cb3b53b898476117588f4e2f9159" + integrity sha512-VoC83HWXmCrF6rgkyxS9GHv8W9Q5nhMKho+OadDJGzL2oDYbYEppBaCMH6pFlwLeqj2QS+hhkw2kpXkSdD1JxQ== + dependencies: + sprintf-js "^1.1.1" + util-deprecate "^1.0.2" + +unherit@^1.0.4: + version "1.1.3" + resolved "https://registry.npmmirror.com/unherit/-/unherit-1.1.3.tgz#6c9b503f2b41b262330c80e91c8614abdaa69c22" + integrity sha512-Ft16BJcnapDKp0+J/rqFC3Rrk6Y/Ng4nzsC028k2jdDII/rdZ7Wd3pPT/6+vIIxRagwRc9K0IUX0Ra4fKvw+WQ== + dependencies: + inherits "^2.0.0" + xtend "^4.0.0" + +unicode-canonical-property-names-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz#301acdc525631670d39f6146e0e77ff6bbdebddc" + integrity sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ== + +unicode-match-property-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz#54fd16e0ecb167cf04cf1f756bdcc92eba7976c3" + integrity sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q== + dependencies: + unicode-canonical-property-names-ecmascript "^2.0.0" + unicode-property-aliases-ecmascript "^2.0.0" + +unicode-match-property-value-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.0.0.tgz#1a01aa57247c14c568b89775a54938788189a714" + integrity sha512-7Yhkc0Ye+t4PNYzOGKedDhXbYIBe1XEQYQxOPyhcXNMJ0WCABqqj6ckydd6pWRZTHV4GuCPKdBAUiMc60tsKVw== + +unicode-property-aliases-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.0.0.tgz#0a36cb9a585c4f6abd51ad1deddb285c165297c8" + integrity sha512-5Zfuy9q/DFr4tfO7ZPeVXb1aPoeQSdeFMLpYuFebehDAhbuevLs5yxSZmIFN1tP5F9Wl4IpJrYojg85/zgyZHQ== + +unified@^9.1.0, unified@^9.2.2: + version "9.2.2" + resolved "https://registry.npmmirror.com/unified/-/unified-9.2.2.tgz#67649a1abfc3ab85d2969502902775eb03146975" + integrity sha512-Sg7j110mtefBD+qunSLO1lqOEKdrwBFBrR6Qd8f4uwkhWNlbkaqwHse6e7QvD3AP/MNoJdEDLaf8OxYyoWgorQ== + dependencies: + bail "^1.0.0" + extend "^3.0.0" + is-buffer "^2.0.0" + is-plain-obj "^2.0.0" + trough "^1.0.0" + vfile "^4.0.0" + +unique-string@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unique-string/-/unique-string-2.0.0.tgz#39c6451f81afb2749de2b233e3f7c5e8843bd89d" + integrity sha512-uNaeirEPvpZWSgzwsPGtU2zVSTrn/8L5q/IexZmH0eH6SA73CmAA5U4GwORTxQAZs95TAXLNqeLoPPNO5gZfWg== + dependencies: + crypto-random-string "^2.0.0" + +unist-builder@^2.0.0: + version "2.0.3" + resolved "https://registry.npmmirror.com/unist-builder/-/unist-builder-2.0.3.tgz#77648711b5d86af0942f334397a33c5e91516436" + integrity sha512-f98yt5pnlMWlzP539tPc4grGMsFaQQlP/vM396b00jngsiINumNmsY8rkXjfoi1c6QaM8nQ3vaGDuoKWbe/1Uw== + +unist-util-generated@^1.0.0: + version "1.1.6" + resolved "https://registry.npmmirror.com/unist-util-generated/-/unist-util-generated-1.1.6.tgz#5ab51f689e2992a472beb1b35f2ce7ff2f324d4b" + integrity sha512-cln2Mm1/CZzN5ttGK7vkoGw+RZ8VcUH6BtGbq98DDtRGquAAOXig1mrBQYelOwMXYS8rK+vZDyyojSjp7JX+Lg== + +unist-util-is@^4.0.0: + version "4.1.0" + resolved "https://registry.npmmirror.com/unist-util-is/-/unist-util-is-4.1.0.tgz#976e5f462a7a5de73d94b706bac1b90671b57797" + integrity sha512-ZOQSsnce92GrxSqlnEEseX0gi7GH9zTJZ0p9dtu87WRb/37mMPO2Ilx1s/t9vBHrFhbgweUwb+t7cIn5dxPhZg== + +unist-util-modify-children@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/unist-util-modify-children/-/unist-util-modify-children-2.0.0.tgz#9c9c30d4e32502aabb3fde10d7872a17c86801e2" + integrity sha512-HGrj7JQo9DwZt8XFsX8UD4gGqOsIlCih9opG6Y+N11XqkBGKzHo8cvDi+MfQQgiZ7zXRUiQREYHhjOBHERTMdg== + dependencies: + array-iterate "^1.0.0" + +unist-util-position@^3.0.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/unist-util-position/-/unist-util-position-3.1.0.tgz#1c42ee6301f8d52f47d14f62bbdb796571fa2d47" + integrity sha512-w+PkwCbYSFw8vpgWD0v7zRCl1FpY3fjDSQ3/N/wNd9Ffa4gPi8+4keqt99N3XW6F99t/mUzp2xAhNmfKWp95QA== + +unist-util-remove-position@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/unist-util-remove-position/-/unist-util-remove-position-3.0.0.tgz#4cd19e82c8e665f462b6acfcfd0a8353235a88e9" + integrity sha512-17kIOuolVuK16LMb9KyMJlqdfCtlfQY5FjY3Sdo9iC7F5wqdXhNjMq0PBvMpkVNNnAmHxXssUW+rZ9T2zbP0Rg== + dependencies: + unist-util-visit "^2.0.0" + +unist-util-select@^3.0.4: + version "3.0.4" + resolved "https://registry.npmmirror.com/unist-util-select/-/unist-util-select-3.0.4.tgz#702c9dc1db1b2bbbfe27f796fce99e43f25edc60" + integrity sha512-xf1zCu4okgPqGLdhCDpRnjwBNyv3EqjiXRUbz2SdK1+qnLMB7uXXajfzuBvvbHoQ+JLyp4AEbFCGndmc6S72sw== + dependencies: + css-selector-parser "^1.0.0" + not "^0.1.0" + nth-check "^2.0.0" + unist-util-is "^4.0.0" + zwitch "^1.0.0" + +unist-util-stringify-position@^2.0.0: + version "2.0.3" + resolved "https://registry.npmmirror.com/unist-util-stringify-position/-/unist-util-stringify-position-2.0.3.tgz#cce3bfa1cdf85ba7375d1d5b17bdc4cada9bd9da" + integrity sha512-3faScn5I+hy9VleOq/qNbAd6pAx7iH5jYBMS9I1HgQVijz/4mv5Bvw5iw1sC/90CODiKo81G/ps8AJrISn687g== + dependencies: + "@types/unist" "^2.0.2" + +unist-util-visit-children@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/unist-util-visit-children/-/unist-util-visit-children-1.1.4.tgz#e8a087e58a33a2815f76ea1901c15dec2cb4b432" + integrity sha512-sA/nXwYRCQVRwZU2/tQWUqJ9JSFM1X3x7JIOsIgSzrFHcfVt6NkzDtKzyxg2cZWkCwGF9CO8x4QNZRJRMK8FeQ== + +unist-util-visit-parents@^3.0.0, unist-util-visit-parents@^3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/unist-util-visit-parents/-/unist-util-visit-parents-3.1.1.tgz#65a6ce698f78a6b0f56aa0e88f13801886cdaef6" + integrity sha512-1KROIZWo6bcMrZEwiH2UrXDyalAa0uqzWCxCJj6lPOvTve2WkfgCytoDTPaMnodXh1WrXOq0haVYHj99ynJlsg== + dependencies: + "@types/unist" "^2.0.0" + unist-util-is "^4.0.0" + +unist-util-visit@^2.0.0, unist-util-visit@^2.0.3: + version "2.0.3" + resolved "https://registry.npmmirror.com/unist-util-visit/-/unist-util-visit-2.0.3.tgz#c3703893146df47203bb8a9795af47d7b971208c" + integrity sha512-iJ4/RczbJMkD0712mGktuGpm/U4By4FfDonL7N/9tATGIF4imikjOuagyMY53tnZq3NP6BcmlrHhEKAfGWjh7Q== + dependencies: + "@types/unist" "^2.0.0" + unist-util-is "^4.0.0" + unist-util-visit-parents "^3.0.0" + +universalify@^0.1.0: + version "0.1.2" + resolved "https://registry.npmmirror.com/universalify/-/universalify-0.1.2.tgz#b646f69be3942dabcecc9d6639c80dc105efaa66" + integrity sha512-rBJeI5CXAlmy1pV+617WB9J63U6XcazHHF2f2dbJix4XzpUF0RS3Zbj0FGIOCAva5P/d/GBOYaACQ1w+0azUkg== + +universalify@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/universalify/-/universalify-2.0.0.tgz#75a4984efedc4b08975c5aeb73f530d02df25717" + integrity sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ== + +unixify@1.0.0, unixify@^1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/unixify/-/unixify-1.0.0.tgz#3a641c8c2ffbce4da683a5c70f03a462940c2090" + integrity sha512-6bc58dPYhCMHHuwxldQxO3RRNZ4eCogZ/st++0+fcC1nr0jiGUtAdBJ2qzmLQWSxbtz42pWt4QQMiZ9HvZf5cg== + dependencies: + normalize-path "^2.1.1" + +unpipe@1.0.0, unpipe@~1.0.0: + version "1.0.0" + resolved "https://registry.npmmirror.com/unpipe/-/unpipe-1.0.0.tgz#b2bf4ee8514aae6165b4817829d21b2ef49904ec" + integrity sha512-pjy2bYhSsufwWlKwPc+l3cN7+wuJlK6uz0YdJEOlQDbl6jo/YlPi4mb8agUkVC8BF7V8NuzeyPNqRksA3hztKQ== + +unquote@~1.1.1: + version "1.1.1" + resolved "https://registry.npmmirror.com/unquote/-/unquote-1.1.1.tgz#8fded7324ec6e88a0ff8b905e7c098cdc086d544" + integrity sha512-vRCqFv6UhXpWxZPyGDh/F3ZpNv8/qo7w6iufLpQg9aKnQ71qM4B5KiI7Mia9COcjEhrO9LueHpMYjYzsWH3OIg== + +update-notifier@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/update-notifier/-/update-notifier-5.1.0.tgz#4ab0d7c7f36a231dd7316cf7729313f0214d9ad9" + integrity sha512-ItnICHbeMh9GqUy31hFPrD1kcuZ3rpxDZbf4KUDavXwS0bW5m7SLbDQpGX3UYr072cbrF5hFUs3r5tUsPwjfHw== + dependencies: + boxen "^5.0.0" + chalk "^4.1.0" + configstore "^5.0.1" + has-yarn "^2.1.0" + import-lazy "^2.1.0" + is-ci "^2.0.0" + is-installed-globally "^0.4.0" + is-npm "^5.0.0" + is-yarn-global "^0.3.0" + latest-version "^5.1.0" + pupa "^2.1.1" + semver "^7.3.4" + semver-diff "^3.1.1" + xdg-basedir "^4.0.0" + +upper-case-first@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/upper-case-first/-/upper-case-first-2.0.2.tgz#992c3273f882abd19d1e02894cc147117f844324" + integrity sha512-514ppYHBaKwfJRK/pNC6c/OxfGa0obSnAl106u97Ed0I625Nin96KAjttZF6ZL3e1XLtphxnqrOi9iWgm+u+bg== + dependencies: + tslib "^2.0.3" + +upper-case@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/upper-case/-/upper-case-2.0.2.tgz#d89810823faab1df1549b7d97a76f8662bae6f7a" + integrity sha512-KgdgDGJt2TpuwBUIjgG6lzw2GWFRCW9Qkfkiv0DxqHHLYJHmtmdUIKcZd8rHgFSjopVTlw6ggzCm1b8MFQwikg== + dependencies: + tslib "^2.0.3" + +uri-js@^4.2.2: + version "4.4.1" + resolved "https://registry.npmmirror.com/uri-js/-/uri-js-4.4.1.tgz#9b1a52595225859e55f669d928f88c6c57f2a77e" + integrity sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg== + dependencies: + punycode "^2.1.0" + +urix@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/urix/-/urix-0.1.0.tgz#da937f7a62e21fec1fd18d49b35c2935067a6c72" + integrity sha512-Am1ousAhSLBeB9cG/7k7r2R0zj50uDRlZHPGbazid5s9rlF1F/QKYObEKSIunSjIOkJZqwRRLpvewjEkM7pSqg== + +url-loader@^4.1.1: + version "4.1.1" + resolved "https://registry.npmmirror.com/url-loader/-/url-loader-4.1.1.tgz#28505e905cae158cf07c92ca622d7f237e70a4e2" + integrity sha512-3BTV812+AVHHOJQO8O5MkWgZ5aosP7GnROJwvzLS9hWDj00lZ6Z0wNak423Lp9PBZN05N+Jk/N5Si8jRAlGyWA== + dependencies: + loader-utils "^2.0.0" + mime-types "^2.1.27" + schema-utils "^3.0.0" + +url-parse-lax@^3.0.0: + version "3.0.0" + resolved "https://registry.npmmirror.com/url-parse-lax/-/url-parse-lax-3.0.0.tgz#16b5cafc07dbe3676c1b1999177823d6503acb0c" + integrity sha512-NjFKA0DidqPa5ciFcSrXnAltTtzz84ogy+NebPvfEgAck0+TNg4UJ4IN+fB7zRZfbgUf0syOo9MDxFkDSMuFaQ== + dependencies: + prepend-http "^2.0.0" + +use-composed-ref@^1.3.0: + version "1.3.0" + resolved "https://registry.npmmirror.com/use-composed-ref/-/use-composed-ref-1.3.0.tgz#3d8104db34b7b264030a9d916c5e94fbe280dbda" + integrity sha512-GLMG0Jc/jiKov/3Ulid1wbv3r54K9HlMW29IWcDFPEqFkSO2nS0MuefWgMJpeHQ9YJeXDL3ZUF+P3jdXlZX/cQ== + +use-isomorphic-layout-effect@^1.1.1: + version "1.1.2" + resolved "https://registry.npmmirror.com/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.1.2.tgz#497cefb13d863d687b08477d9e5a164ad8c1a6fb" + integrity sha512-49L8yCO3iGT/ZF9QttjwLF/ZD9Iwto5LnH5LmEdk/6cFmXddqi2ulF0edxTwjj+7mqvpVVGQWvbXZdn32wRSHA== + +use-latest@^1.2.1: + version "1.2.1" + resolved "https://registry.npmmirror.com/use-latest/-/use-latest-1.2.1.tgz#d13dfb4b08c28e3e33991546a2cee53e14038cf2" + integrity sha512-xA+AVm/Wlg3e2P/JiItTziwS7FK92LWrDB0p+hgXloIMuVCeJJ8v6f0eeHyPZaJrM+usM1FkFfbNCrJGs8A/zw== + dependencies: + use-isomorphic-layout-effect "^1.1.1" + +utif@^2.0.1: + version "2.0.1" + resolved "https://registry.npmmirror.com/utif/-/utif-2.0.1.tgz#9e1582d9bbd20011a6588548ed3266298e711759" + integrity sha512-Z/S1fNKCicQTf375lIP9G8Sa1H/phcysstNrrSdZKj1f9g58J4NMgb5IgiEZN9/nLMPDwF0W7hdOe9Qq2IYoLg== + dependencies: + pako "^1.0.5" + +util-deprecate@^1.0.1, util-deprecate@^1.0.2, util-deprecate@~1.0.1: + version "1.0.2" + resolved "https://registry.npmmirror.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf" + integrity sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw== + +util.promisify@~1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/util.promisify/-/util.promisify-1.0.1.tgz#6baf7774b80eeb0f7520d8b81d07982a59abbaee" + integrity sha512-g9JpC/3He3bm38zsLupWryXHoEcS22YHthuPQSJdMy6KNrzIRzWqcsHzD/WUnqe45whVou4VIsPew37DoXWNrA== + dependencies: + define-properties "^1.1.3" + es-abstract "^1.17.2" + has-symbols "^1.0.1" + object.getownpropertydescriptors "^2.1.0" + +utila@~0.4: + version "0.4.0" + resolved "https://registry.npmmirror.com/utila/-/utila-0.4.0.tgz#8a16a05d445657a3aea5eecc5b12a4fa5379772c" + integrity sha512-Z0DbgELS9/L/75wZbro8xAnT50pBVFQZ+hUEueGDU5FN51YSCYM+jdxsfCiHjwNP/4LCDD0i/graKpeBnOXKRA== + +utility-types@^3.10.0: + version "3.10.0" + resolved "https://registry.npmmirror.com/utility-types/-/utility-types-3.10.0.tgz#ea4148f9a741015f05ed74fd615e1d20e6bed82b" + integrity sha512-O11mqxmi7wMKCo6HKFt5AhO4BwY3VV68YU07tgxfz8zJTIxr4BpsezN49Ffwy9j3ZpwwJp4fkRwjRzq3uWE6Rg== + +utils-merge@1.0.1: + version "1.0.1" + resolved "https://registry.npmmirror.com/utils-merge/-/utils-merge-1.0.1.tgz#9f95710f50a267947b2ccc124741c1028427e713" + integrity sha512-pMZTvIkT1d+TFGvDOqodOclx0QWkkgi6Tdoa8gC8ffGAAqz9pzPTZWAybbsHHoED/ztMtkv/VoYTYyShUn81hA== + +uuid@^3.3.2: + version "3.4.0" + resolved "https://registry.npmmirror.com/uuid/-/uuid-3.4.0.tgz#b23e4358afa8a202fe7a100af1f5f883f02007ee" + integrity sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A== + +uuid@^8.3.2: + version "8.3.2" + resolved "https://registry.npmmirror.com/uuid/-/uuid-8.3.2.tgz#80d5b5ced271bb9af6c445f21a1a04c606cefbe2" + integrity sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg== + +v8-compile-cache@^2.0.3: + version "2.3.0" + resolved "https://registry.npmmirror.com/v8-compile-cache/-/v8-compile-cache-2.3.0.tgz#2de19618c66dc247dcfb6f99338035d8245a2cee" + integrity sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA== + +valid-url@1.0.9, valid-url@^1.0.9: + version "1.0.9" + resolved "https://registry.npmmirror.com/valid-url/-/valid-url-1.0.9.tgz#1c14479b40f1397a75782f115e4086447433a200" + integrity sha512-QQDsV8OnSf5Uc30CKSwG9lnhMPe6exHtTXLRYX8uMwKENy640pU+2BgBL0LRbDh/eYRahNCS7aewCx0wf3NYVA== + +validate-npm-package-license@^3.0.1: + version "3.0.4" + resolved "https://registry.npmmirror.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz#fc91f6b9c7ba15c857f4cb2c5defeec39d4f410a" + integrity sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew== + dependencies: + spdx-correct "^3.0.0" + spdx-expression-parse "^3.0.0" + +value-or-promise@1.0.11: + version "1.0.11" + resolved "https://registry.npmmirror.com/value-or-promise/-/value-or-promise-1.0.11.tgz#3e90299af31dd014fe843fe309cefa7c1d94b140" + integrity sha512-41BrgH+dIbCFXClcSapVs5M6GkENd3gQOJpEfPDNa71LsUGMXDL0jMWpI/Rh7WhX+Aalfz2TTS3Zt5pUsbnhLg== + +value-or-promise@1.0.6: + version "1.0.6" + resolved "https://registry.npmmirror.com/value-or-promise/-/value-or-promise-1.0.6.tgz#218aa4794aa2ee24dcf48a29aba4413ed584747f" + integrity sha512-9r0wQsWD8z/BxPOvnwbPf05ZvFngXyouE9EKB+5GbYix+BYnAwrIChCUyFIinfbf2FL/U71z+CPpbnmTdxrwBg== + +vary@^1, vary@~1.1.2: + version "1.1.2" + resolved "https://registry.npmmirror.com/vary/-/vary-1.1.2.tgz#2299f02c6ded30d4a5961b0b9f74524a18f634fc" + integrity sha512-BNGbWLfd0eUPabhkXUVm0j8uuvREyTh5ovRa/dyow/BqAbZJyC+5fU+IzQOzmAKzYqYRAISoRhdQr3eIZ/PXqg== + +verror@1.10.0: + version "1.10.0" + resolved "https://registry.npmmirror.com/verror/-/verror-1.10.0.tgz#3a105ca17053af55d6e270c1f8288682e18da400" + integrity sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw== + dependencies: + assert-plus "^1.0.0" + core-util-is "1.0.2" + extsprintf "^1.2.0" + +vfile-location@^3.1.0, vfile-location@^3.2.0: + version "3.2.0" + resolved "https://registry.npmmirror.com/vfile-location/-/vfile-location-3.2.0.tgz#d8e41fbcbd406063669ebf6c33d56ae8721d0f3c" + integrity sha512-aLEIZKv/oxuCDZ8lkJGhuhztf/BW4M+iHdCwglA/eWc+vtuRFJj8EtgceYFX4LRjOhCAAiNHsKGssC6onJ+jbA== + +vfile-message@^2.0.0: + version "2.0.4" + resolved "https://registry.npmmirror.com/vfile-message/-/vfile-message-2.0.4.tgz#5b43b88171d409eae58477d13f23dd41d52c371a" + integrity sha512-DjssxRGkMvifUOJre00juHoP9DPWuzjxKuMDrhNbk2TdaYYBNMStsNhEOt3idrtI12VQYM/1+iM0KOzXi4pxwQ== + dependencies: + "@types/unist" "^2.0.0" + unist-util-stringify-position "^2.0.0" + +vfile@^4.0.0: + version "4.2.1" + resolved "https://registry.npmmirror.com/vfile/-/vfile-4.2.1.tgz#03f1dce28fc625c625bc6514350fbdb00fa9e624" + integrity sha512-O6AE4OskCG5S1emQ/4gl8zK586RqA3srz3nfK/Viy0UPToBc5Trp9BVFb1u0CjsKrAWwnpr4ifM/KBXPWwJbCA== + dependencies: + "@types/unist" "^2.0.0" + is-buffer "^2.0.0" + unist-util-stringify-position "^2.0.0" + vfile-message "^2.0.0" + +warning@^4.0.2: + version "4.0.3" + resolved "https://registry.npmmirror.com/warning/-/warning-4.0.3.tgz#16e9e077eb8a86d6af7d64aa1e05fd85b4678ca3" + integrity sha512-rpJyN222KWIvHJ/F53XSZv0Zl/accqHR8et1kpaMTD/fLCRxtV8iX8czMzY7sVZupTI3zcUTg8eycS2kNF9l6w== + dependencies: + loose-envify "^1.0.0" + +watchpack@^2.3.1: + version "2.4.0" + resolved "https://registry.npmmirror.com/watchpack/-/watchpack-2.4.0.tgz#fa33032374962c78113f93c7f2fb4c54c9862a5d" + integrity sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg== + dependencies: + glob-to-regexp "^0.4.1" + graceful-fs "^4.1.2" + +weak-lru-cache@^1.2.2: + version "1.2.2" + resolved "https://registry.npmmirror.com/weak-lru-cache/-/weak-lru-cache-1.2.2.tgz#fdbb6741f36bae9540d12f480ce8254060dccd19" + integrity sha512-DEAoo25RfSYMuTGc9vPJzZcZullwIqRDSI9LOy+fkCJPi6hykCnfKaXTuPBDuXAUcqHXyOgFtHNp/kB2FjYHbw== + +web-namespaces@^1.0.0: + version "1.1.4" + resolved "https://registry.npmmirror.com/web-namespaces/-/web-namespaces-1.1.4.tgz#bc98a3de60dadd7faefc403d1076d529f5e030ec" + integrity sha512-wYxSGajtmoP4WxfejAPIr4l0fVh+jeMXZb08wNc0tMg6xsfZXj3cECqIK0G7ZAqUq0PP8WlMDtaOGVBTAWztNw== + +webidl-conversions@^3.0.0: + version "3.0.1" + resolved "https://registry.npmmirror.com/webidl-conversions/-/webidl-conversions-3.0.1.tgz#24534275e2a7bc6be7bc86611cc16ae0a5654871" + integrity sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ== + +webpack-assets-manifest@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/webpack-assets-manifest/-/webpack-assets-manifest-5.1.0.tgz#5af328f6c8fa760cb9a62af631a83da2b478b791" + integrity sha512-kPuTMEjBrqZQVJ5M6yXNBCEdFbQQn7p+loNXt8NOeDFaAbsNFWqqwR0YL1mfG5LbwhK5FLXWXpuK3GuIIZ46rg== + dependencies: + chalk "^4.0" + deepmerge "^4.0" + lockfile "^1.0" + lodash.get "^4.0" + lodash.has "^4.0" + schema-utils "^3.0" + tapable "^2.0" + +webpack-dev-middleware@^4.3.0: + version "4.3.0" + resolved "https://registry.npmmirror.com/webpack-dev-middleware/-/webpack-dev-middleware-4.3.0.tgz#179cc40795882cae510b1aa7f3710cbe93c9333e" + integrity sha512-PjwyVY95/bhBh6VUqt6z4THplYcsvQ8YNNBTBM873xLVmw8FLeALn0qurHbs9EmcfhzQis/eoqypSnZeuUz26w== + dependencies: + colorette "^1.2.2" + mem "^8.1.1" + memfs "^3.2.2" + mime-types "^2.1.30" + range-parser "^1.2.1" + schema-utils "^3.0.0" + +webpack-merge@^5.8.0: + version "5.8.0" + resolved "https://registry.npmmirror.com/webpack-merge/-/webpack-merge-5.8.0.tgz#2b39dbf22af87776ad744c390223731d30a68f61" + integrity sha512-/SaI7xY0831XwP6kzuwhKWVKDP9t1QY1h65lAFLbZqMPIuYcD9QAW4u9STIbU9kaJbPBB/geU/gLr1wDjOhQ+Q== + dependencies: + clone-deep "^4.0.1" + wildcard "^2.0.0" + +webpack-sources@^1.1.0: + version "1.4.3" + resolved "https://registry.npmmirror.com/webpack-sources/-/webpack-sources-1.4.3.tgz#eedd8ec0b928fbf1cbfe994e22d2d890f330a933" + integrity sha512-lgTS3Xhv1lCOKo7SA5TjKXMjpSM4sBjNV5+q2bqesbSPs5FjGmU6jjtBSkX9b4qW87vDIsCIlUPOEhbZrMdjeQ== + dependencies: + source-list-map "^2.0.0" + source-map "~0.6.1" + +webpack-sources@^3.2.3: + version "3.2.3" + resolved "https://registry.npmmirror.com/webpack-sources/-/webpack-sources-3.2.3.tgz#2d4daab8451fd4b240cc27055ff6a0c2ccea0cde" + integrity sha512-/DyMEOrDgLKKIG0fmvtz+4dUX/3Ghozwgm6iPp8KRhvn+eQf9+Q7GWxVNMk3+uCPWfdXYC4ExGBckIXdFEfH1w== + +webpack-stats-plugin@^1.0.3: + version "1.0.3" + resolved "https://registry.npmmirror.com/webpack-stats-plugin/-/webpack-stats-plugin-1.0.3.tgz#0f64551a0b984b48a9e7acdee32e3cfda556fe51" + integrity sha512-tV/SQHl6lKfBahJcNDmz8JG1rpWPB9NEDQSMIoL74oVAotdxYljpgIsgLzgc1N9QrtA9KEA0moJVwQtNZv2aDA== + +webpack-virtual-modules@^0.3.2: + version "0.3.2" + resolved "https://registry.npmmirror.com/webpack-virtual-modules/-/webpack-virtual-modules-0.3.2.tgz#b7baa30971a22d99451f897db053af48ec29ad2c" + integrity sha512-RXQXioY6MhzM4CNQwmBwKXYgBs6ulaiQ8bkNQEl2J6Z+V+s7lgl/wGvaI/I0dLnYKB8cKsxQc17QOAVIphPLDw== + dependencies: + debug "^3.0.0" + +webpack@^5.61.0: + version "5.73.0" + resolved "https://registry.npmmirror.com/webpack/-/webpack-5.73.0.tgz#bbd17738f8a53ee5760ea2f59dce7f3431d35d38" + integrity sha512-svjudQRPPa0YiOYa2lM/Gacw0r6PvxptHj4FuEKQ2kX05ZLkjbVc5MnPs6its5j7IZljnIqSVo/OsY2X0IpHGA== + dependencies: + "@types/eslint-scope" "^3.7.3" + "@types/estree" "^0.0.51" + "@webassemblyjs/ast" "1.11.1" + "@webassemblyjs/wasm-edit" "1.11.1" + "@webassemblyjs/wasm-parser" "1.11.1" + acorn "^8.4.1" + acorn-import-assertions "^1.7.6" + browserslist "^4.14.5" + chrome-trace-event "^1.0.2" + enhanced-resolve "^5.9.3" + es-module-lexer "^0.9.0" + eslint-scope "5.1.1" + events "^3.2.0" + glob-to-regexp "^0.4.1" + graceful-fs "^4.2.9" + json-parse-even-better-errors "^2.3.1" + loader-runner "^4.2.0" + mime-types "^2.1.27" + neo-async "^2.6.2" + schema-utils "^3.1.0" + tapable "^2.1.1" + terser-webpack-plugin "^5.1.3" + watchpack "^2.3.1" + webpack-sources "^3.2.3" + +whatwg-url@^5.0.0: + version "5.0.0" + resolved "https://registry.npmmirror.com/whatwg-url/-/whatwg-url-5.0.0.tgz#966454e8765462e37644d3626f6742ce8b70965d" + integrity sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw== + dependencies: + tr46 "~0.0.3" + webidl-conversions "^3.0.0" + +which-boxed-primitive@^1.0.2: + version "1.0.2" + resolved "https://registry.npmmirror.com/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz#13757bc89b209b049fe5d86430e21cf40a89a8e6" + integrity sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg== + dependencies: + is-bigint "^1.0.1" + is-boolean-object "^1.1.0" + is-number-object "^1.0.4" + is-string "^1.0.5" + is-symbol "^1.0.3" + +which-module@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/which-module/-/which-module-2.0.0.tgz#d9ef07dce77b9902b8a3a8fa4b31c3e3f7e6e87a" + integrity sha512-B+enWhmw6cjfVC7kS8Pj9pCrKSc5txArRyaYGe088shv/FGWH+0Rjx/xPgtsWfsUtS27FkP697E4DDhgrgoc0Q== + +which@^1.2.9, which@^1.3.1: + version "1.3.1" + resolved "https://registry.npmmirror.com/which/-/which-1.3.1.tgz#a45043d54f5805316da8d62f9f50918d3da70b0a" + integrity sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ== + dependencies: + isexe "^2.0.0" + +which@^2.0.1, which@^2.0.2: + version "2.0.2" + resolved "https://registry.npmmirror.com/which/-/which-2.0.2.tgz#7c6a8dd0a636a0327e10b59c9286eee93f3f51b1" + integrity sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA== + dependencies: + isexe "^2.0.0" + +wide-align@^1.1.0: + version "1.1.5" + resolved "https://registry.npmmirror.com/wide-align/-/wide-align-1.1.5.tgz#df1d4c206854369ecf3c9a4898f1b23fbd9d15d3" + integrity sha512-eDMORYaPNZ4sQIuuYPDHdQvf4gyCF9rEEV/yPxGfwPkRodwEgiMUUXTx/dex+Me0wxx53S+NgUHaP7y3MGlDmg== + dependencies: + string-width "^1.0.2 || 2 || 3 || 4" + +widest-line@^3.1.0: + version "3.1.0" + resolved "https://registry.npmmirror.com/widest-line/-/widest-line-3.1.0.tgz#8292333bbf66cb45ff0de1603b136b7ae1496eca" + integrity sha512-NsmoXalsWVDMGupxZ5R08ka9flZjjiLvHVAWYOKtiKM8ujtZWr9cRffak+uSE48+Ob8ObalXpwyeUiyDD6QFgg== + dependencies: + string-width "^4.0.0" + +wildcard@^2.0.0: + version "2.0.0" + resolved "https://registry.npmmirror.com/wildcard/-/wildcard-2.0.0.tgz#a77d20e5200c6faaac979e4b3aadc7b3dd7f8fec" + integrity sha512-JcKqAHLPxcdb9KM49dufGXn2x3ssnfjbcaQdLlfZsL9rH9wgDQjUtDxbo8NE0F6SFvydeu1VhZe7hZuHsB2/pw== + +word-wrap@^1.2.3: + version "1.2.3" + resolved "https://registry.npmmirror.com/word-wrap/-/word-wrap-1.2.3.tgz#610636f6b1f703891bd34771ccb17fb93b47079c" + integrity sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ== + +workbox-background-sync@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-background-sync/-/workbox-background-sync-4.3.1.tgz#26821b9bf16e9e37fd1d640289edddc08afd1950" + integrity sha512-1uFkvU8JXi7L7fCHVBEEnc3asPpiAL33kO495UMcD5+arew9IbKW2rV5lpzhoWcm/qhGB89YfO4PmB/0hQwPRg== + dependencies: + workbox-core "^4.3.1" + +workbox-broadcast-update@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-broadcast-update/-/workbox-broadcast-update-4.3.1.tgz#e2c0280b149e3a504983b757606ad041f332c35b" + integrity sha512-MTSfgzIljpKLTBPROo4IpKjESD86pPFlZwlvVG32Kb70hW+aob4Jxpblud8EhNb1/L5m43DUM4q7C+W6eQMMbA== + dependencies: + workbox-core "^4.3.1" + +workbox-build@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-build/-/workbox-build-4.3.1.tgz#414f70fb4d6de47f6538608b80ec52412d233e64" + integrity sha512-UHdwrN3FrDvicM3AqJS/J07X0KXj67R8Cg0waq1MKEOqzo89ap6zh6LmaLnRAjpB+bDIz+7OlPye9iii9KBnxw== + dependencies: + "@babel/runtime" "^7.3.4" + "@hapi/joi" "^15.0.0" + common-tags "^1.8.0" + fs-extra "^4.0.2" + glob "^7.1.3" + lodash.template "^4.4.0" + pretty-bytes "^5.1.0" + stringify-object "^3.3.0" + strip-comments "^1.0.2" + workbox-background-sync "^4.3.1" + workbox-broadcast-update "^4.3.1" + workbox-cacheable-response "^4.3.1" + workbox-core "^4.3.1" + workbox-expiration "^4.3.1" + workbox-google-analytics "^4.3.1" + workbox-navigation-preload "^4.3.1" + workbox-precaching "^4.3.1" + workbox-range-requests "^4.3.1" + workbox-routing "^4.3.1" + workbox-strategies "^4.3.1" + workbox-streams "^4.3.1" + workbox-sw "^4.3.1" + workbox-window "^4.3.1" + +workbox-cacheable-response@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-cacheable-response/-/workbox-cacheable-response-4.3.1.tgz#f53e079179c095a3f19e5313b284975c91428c91" + integrity sha512-Rp5qlzm6z8IOvnQNkCdO9qrDgDpoPNguovs0H8C+wswLuPgSzSp9p2afb5maUt9R1uTIwOXrVQMmPfPypv+npw== + dependencies: + workbox-core "^4.3.1" + +workbox-core@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-core/-/workbox-core-4.3.1.tgz#005d2c6a06a171437afd6ca2904a5727ecd73be6" + integrity sha512-I3C9jlLmMKPxAC1t0ExCq+QoAMd0vAAHULEgRZ7kieCdUd919n53WC0AfvokHNwqRhGn+tIIj7vcb5duCjs2Kg== + +workbox-expiration@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-expiration/-/workbox-expiration-4.3.1.tgz#d790433562029e56837f341d7f553c4a78ebe921" + integrity sha512-vsJLhgQsQouv9m0rpbXubT5jw0jMQdjpkum0uT+d9tTwhXcEZks7qLfQ9dGSaufTD2eimxbUOJfWLbNQpIDMPw== + dependencies: + workbox-core "^4.3.1" + +workbox-google-analytics@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-google-analytics/-/workbox-google-analytics-4.3.1.tgz#9eda0183b103890b5c256e6f4ea15a1f1548519a" + integrity sha512-xzCjAoKuOb55CBSwQrbyWBKqp35yg1vw9ohIlU2wTy06ZrYfJ8rKochb1MSGlnoBfXGWss3UPzxR5QL5guIFdg== + dependencies: + workbox-background-sync "^4.3.1" + workbox-core "^4.3.1" + workbox-routing "^4.3.1" + workbox-strategies "^4.3.1" + +workbox-navigation-preload@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-navigation-preload/-/workbox-navigation-preload-4.3.1.tgz#29c8e4db5843803b34cd96dc155f9ebd9afa453d" + integrity sha512-K076n3oFHYp16/C+F8CwrRqD25GitA6Rkd6+qAmLmMv1QHPI2jfDwYqrytOfKfYq42bYtW8Pr21ejZX7GvALOw== + dependencies: + workbox-core "^4.3.1" + +workbox-precaching@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-precaching/-/workbox-precaching-4.3.1.tgz#9fc45ed122d94bbe1f0ea9584ff5940960771cba" + integrity sha512-piSg/2csPoIi/vPpp48t1q5JLYjMkmg5gsXBQkh/QYapCdVwwmKlU9mHdmy52KsDGIjVaqEUMFvEzn2LRaigqQ== + dependencies: + workbox-core "^4.3.1" + +workbox-range-requests@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-range-requests/-/workbox-range-requests-4.3.1.tgz#f8a470188922145cbf0c09a9a2d5e35645244e74" + integrity sha512-S+HhL9+iTFypJZ/yQSl/x2Bf5pWnbXdd3j57xnb0V60FW1LVn9LRZkPtneODklzYuFZv7qK6riZ5BNyc0R0jZA== + dependencies: + workbox-core "^4.3.1" + +workbox-routing@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-routing/-/workbox-routing-4.3.1.tgz#a675841af623e0bb0c67ce4ed8e724ac0bed0cda" + integrity sha512-FkbtrODA4Imsi0p7TW9u9MXuQ5P4pVs1sWHK4dJMMChVROsbEltuE79fBoIk/BCztvOJ7yUpErMKa4z3uQLX+g== + dependencies: + workbox-core "^4.3.1" + +workbox-strategies@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-strategies/-/workbox-strategies-4.3.1.tgz#d2be03c4ef214c115e1ab29c9c759c9fe3e9e646" + integrity sha512-F/+E57BmVG8dX6dCCopBlkDvvhg/zj6VDs0PigYwSN23L8hseSRwljrceU2WzTvk/+BSYICsWmRq5qHS2UYzhw== + dependencies: + workbox-core "^4.3.1" + +workbox-streams@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-streams/-/workbox-streams-4.3.1.tgz#0b57da70e982572de09c8742dd0cb40a6b7c2cc3" + integrity sha512-4Kisis1f/y0ihf4l3u/+ndMkJkIT4/6UOacU3A4BwZSAC9pQ9vSvJpIi/WFGQRH/uPXvuVjF5c2RfIPQFSS2uA== + dependencies: + workbox-core "^4.3.1" + +workbox-sw@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-sw/-/workbox-sw-4.3.1.tgz#df69e395c479ef4d14499372bcd84c0f5e246164" + integrity sha512-0jXdusCL2uC5gM3yYFT6QMBzKfBr2XTk0g5TPAV4y8IZDyVNDyj1a8uSXy3/XrvkVTmQvLN4O5k3JawGReXr9w== + +workbox-window@^4.3.1: + version "4.3.1" + resolved "https://registry.npmmirror.com/workbox-window/-/workbox-window-4.3.1.tgz#ee6051bf10f06afa5483c9b8dfa0531994ede0f3" + integrity sha512-C5gWKh6I58w3GeSc0wp2Ne+rqVw8qwcmZnQGpjiek8A2wpbxSJb1FdCoQVO+jDJs35bFgo/WETgl1fqgsxN0Hg== + dependencies: + workbox-core "^4.3.1" + +wrap-ansi@^5.1.0: + version "5.1.0" + resolved "https://registry.npmmirror.com/wrap-ansi/-/wrap-ansi-5.1.0.tgz#1fd1f67235d5b6d0fee781056001bfb694c03b09" + integrity sha512-QC1/iN/2/RPVJ5jYK8BGttj5z83LmSKmvbvrXPNCLZSEb32KKVDJDl/MOt2N01qU2H/FkzEa9PKto1BqDjtd7Q== + dependencies: + ansi-styles "^3.2.0" + string-width "^3.0.0" + strip-ansi "^5.0.0" + +wrap-ansi@^6.2.0: + version "6.2.0" + resolved "https://registry.npmmirror.com/wrap-ansi/-/wrap-ansi-6.2.0.tgz#e9393ba07102e6c91a3b221478f0257cd2856e53" + integrity sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + +wrap-ansi@^7.0.0: + version "7.0.0" + resolved "https://registry.npmmirror.com/wrap-ansi/-/wrap-ansi-7.0.0.tgz#67e145cff510a6a6984bdf1152911d69d2eb9e43" + integrity sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + +wrappy@1: + version "1.0.2" + resolved "https://registry.npmmirror.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" + integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== + +write-file-atomic@^3.0.0: + version "3.0.3" + resolved "https://registry.npmmirror.com/write-file-atomic/-/write-file-atomic-3.0.3.tgz#56bd5c5a5c70481cd19c571bd39ab965a5de56e8" + integrity sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q== + dependencies: + imurmurhash "^0.1.4" + is-typedarray "^1.0.0" + signal-exit "^3.0.2" + typedarray-to-buffer "^3.1.5" + +ws@7.4.5: + version "7.4.5" + resolved "https://registry.npmmirror.com/ws/-/ws-7.4.5.tgz#a484dd851e9beb6fdb420027e3885e8ce48986c1" + integrity sha512-xzyu3hFvomRfXKH8vOFMU3OguG6oOvhXMo3xsGy3xWExqaM2dxBbVxuD99O7m3ZUFMvvscsZDqxfgMaRr/Nr1g== + +"ws@^5.2.0 || ^6.0.0 || ^7.0.0": + version "7.5.8" + resolved "https://registry.npmmirror.com/ws/-/ws-7.5.8.tgz#ac2729881ab9e7cbaf8787fe3469a48c5c7f636a" + integrity sha512-ri1Id1WinAX5Jqn9HejiGb8crfRio0Qgu8+MtL36rlTA6RLsMdWt1Az/19A2Qij6uSHUMphEFaTKa4WG+UNHNw== + +ws@~7.4.2: + version "7.4.6" + resolved "https://registry.npmmirror.com/ws/-/ws-7.4.6.tgz#5654ca8ecdeee47c33a9a4bf6d28e2be2980377c" + integrity sha512-YmhHDO4MzaDLB+M9ym/mDA5z0naX8j7SIlT8f8z+I0VtzsRbekxEutHSme7NPS2qE8StCYQNUnfWdXta/Yu85A== + +xdg-basedir@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/xdg-basedir/-/xdg-basedir-4.0.0.tgz#4bc8d9984403696225ef83a1573cbbcb4e79db13" + integrity sha512-PSNhEJDejZYV7h50BohL09Er9VaIefr2LMAf3OEmpCkjOi34eYyQYAXUTjEQtZJTKcF0E2UKTh+osDLsgNim9Q== + +xhr@^2.0.1: + version "2.6.0" + resolved "https://registry.npmmirror.com/xhr/-/xhr-2.6.0.tgz#b69d4395e792b4173d6b7df077f0fc5e4e2b249d" + integrity sha512-/eCGLb5rxjx5e3mF1A7s+pLlR6CGyqWN91fv1JgER5mVWg1MZmlhBvy9kjcsOdRk8RrIujotWyJamfyrp+WIcA== + dependencies: + global "~4.4.0" + is-function "^1.0.1" + parse-headers "^2.0.0" + xtend "^4.0.0" + +xml-parse-from-string@^1.0.0: + version "1.0.1" + resolved "https://registry.npmmirror.com/xml-parse-from-string/-/xml-parse-from-string-1.0.1.tgz#a9029e929d3dbcded169f3c6e28238d95a5d5a28" + integrity sha512-ErcKwJTF54uRzzNMXq2X5sMIy88zJvfN2DmdoQvy7PAFJ+tPRU6ydWuOKNMyfmOjdyBQTFREi60s0Y0SyI0G0g== + +xml2js@^0.4.5: + version "0.4.23" + resolved "https://registry.npmmirror.com/xml2js/-/xml2js-0.4.23.tgz#a0c69516752421eb2ac758ee4d4ccf58843eac66" + integrity sha512-ySPiMjM0+pLDftHgXY4By0uswI3SPKLDw/i3UXbnO8M/p28zqexCUoPmQFrYD+/1BzhGJSs2i1ERWKJAtiLrug== + dependencies: + sax ">=0.6.0" + xmlbuilder "~11.0.0" + +xmlbuilder@~11.0.0: + version "11.0.1" + resolved "https://registry.npmmirror.com/xmlbuilder/-/xmlbuilder-11.0.1.tgz#be9bae1c8a046e76b31127726347d0ad7002beb3" + integrity sha512-fDlsI/kFEx7gLvbecc0/ohLG50fugQp8ryHzMTuW9vSa1GJ0XYWKnhsUx7oie3G98+r56aTQIUB4kht42R3JvA== + +xmlhttprequest-ssl@~1.6.2: + version "1.6.3" + resolved "https://registry.npmmirror.com/xmlhttprequest-ssl/-/xmlhttprequest-ssl-1.6.3.tgz#03b713873b01659dfa2c1c5d056065b27ddc2de6" + integrity sha512-3XfeQE/wNkvrIktn2Kf0869fC0BN6UpydVasGIeSm2B1Llihf7/0UfZM+eCkOw3P7bP4+qPgqhm7ZoxuJtFU0Q== + +xss@^1.0.6: + version "1.0.13" + resolved "https://registry.npmmirror.com/xss/-/xss-1.0.13.tgz#6e48f616128b39f366dfadc57411e1eb5b341c6c" + integrity sha512-clu7dxTm1e8Mo5fz3n/oW3UCXBfV89xZ72jM8yzo1vR/pIS0w3sgB3XV2H8Vm6zfGnHL0FzvLJPJEBhd86/z4Q== + dependencies: + commander "^2.20.3" + cssfilter "0.0.10" + +xstate@^4.26.0, xstate@^4.26.1: + version "4.32.1" + resolved "https://registry.npmmirror.com/xstate/-/xstate-4.32.1.tgz#1a09c808a66072938861a3b4acc5b38460244b70" + integrity sha512-QYUd+3GkXZ8i6qdixnOn28bL3EvA++LONYL/EMWwKlFSh/hiLndJ8YTnz77FDs+JUXcwU7NZJg7qoezoRHc4GQ== + +xtend@^4.0.0, xtend@^4.0.2: + version "4.0.2" + resolved "https://registry.npmmirror.com/xtend/-/xtend-4.0.2.tgz#bb72779f5fa465186b1f438f674fa347fdb5db54" + integrity sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ== + +xtend@~2.1.1: + version "2.1.2" + resolved "https://registry.npmmirror.com/xtend/-/xtend-2.1.2.tgz#6efecc2a4dad8e6962c4901b337ce7ba87b5d28b" + integrity sha512-vMNKzr2rHP9Dp/e1NQFnLQlwlhp9L/LfvnsVdHxN1f+uggyVI3i08uD14GPvCToPkdsRfyPqIyYGmIk58V98ZQ== + dependencies: + object-keys "~0.4.0" + +xxhash-wasm@^0.4.2: + version "0.4.2" + resolved "https://registry.npmmirror.com/xxhash-wasm/-/xxhash-wasm-0.4.2.tgz#752398c131a4dd407b5132ba62ad372029be6f79" + integrity sha512-/eyHVRJQCirEkSZ1agRSCwriMhwlyUcFkXD5TPVSLP+IPzjsqMVzZwdoczLp1SoQU0R3dxz1RpIK+4YNQbCVOA== + +y18n@^4.0.0: + version "4.0.3" + resolved "https://registry.npmmirror.com/y18n/-/y18n-4.0.3.tgz#b5f259c82cd6e336921efd7bfd8bf560de9eeedf" + integrity sha512-JKhqTOwSrqNA1NY5lSztJ1GrBiUodLMmIZuLiDaMRJ+itFd+ABVE8XBjOvIWL+rSqNDC74LCSFmlb/U4UZ4hJQ== + +yallist@^2.0.0, yallist@^2.1.2: + version "2.1.2" + resolved "https://registry.npmmirror.com/yallist/-/yallist-2.1.2.tgz#1c11f9218f076089a47dd512f93c6699a6a81d52" + integrity sha512-ncTzHV7NvsQZkYe1DW7cbDLm0YpzHmZF5r/iyP3ZnQtMiJ+pjzisCiMNI+Sj+xQF5pXhSHxSB3uDbsBTzY/c2A== + +yallist@^4.0.0: + version "4.0.0" + resolved "https://registry.npmmirror.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" + integrity sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A== + +yaml-loader@^0.6.0: + version "0.6.0" + resolved "https://registry.npmmirror.com/yaml-loader/-/yaml-loader-0.6.0.tgz#fe1c48b9f4803dace55a59a1474e790ba6ab1b48" + integrity sha512-1bNiLelumURyj+zvVHOv8Y3dpCri0F2S+DCcmps0pA1zWRLjS+FhZQg4o3aUUDYESh73+pKZNI18bj7stpReow== + dependencies: + loader-utils "^1.4.0" + yaml "^1.8.3" + +yaml@^1.10.0, yaml@^1.10.2, yaml@^1.7.2, yaml@^1.8.3: + version "1.10.2" + resolved "https://registry.npmmirror.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b" + integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg== + +yargs-parser@^13.1.2: + version "13.1.2" + resolved "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-13.1.2.tgz#130f09702ebaeef2650d54ce6e3e5706f7a4fb38" + integrity sha512-3lbsNRf/j+A4QuSZfDRA7HRSfWrzO0YjqTJd5kjAq37Zep1CEgaYmrH9Q3GwPiB9cHyd1Y1UwggGhJGoxipbzg== + dependencies: + camelcase "^5.0.0" + decamelize "^1.2.0" + +yargs-parser@^18.1.2: + version "18.1.3" + resolved "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-18.1.3.tgz#be68c4975c6b2abf469236b0c870362fab09a7b0" + integrity sha512-o50j0JeToy/4K6OZcaQmW6lyXXKhq7csREXcDwk2omFPJEwUNOVtJKvmDr9EI1fAJZUyZcRF7kxGBWmRXudrCQ== + dependencies: + camelcase "^5.0.0" + decamelize "^1.2.0" + +yargs-parser@^20.2.3: + version "20.2.9" + resolved "https://registry.npmmirror.com/yargs-parser/-/yargs-parser-20.2.9.tgz#2eb7dc3b0289718fc295f362753845c41a0c94ee" + integrity sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w== + +yargs@^13.3.2: + version "13.3.2" + resolved "https://registry.npmmirror.com/yargs/-/yargs-13.3.2.tgz#ad7ffefec1aa59565ac915f82dccb38a9c31a2dd" + integrity sha512-AX3Zw5iPruN5ie6xGRIDgqkT+ZhnRlZMLMHAs8tg7nRruy2Nb+i5o9bwghAogtM08q1dpr2LVoS8KSTMYpWXUw== + dependencies: + cliui "^5.0.0" + find-up "^3.0.0" + get-caller-file "^2.0.1" + require-directory "^2.1.1" + require-main-filename "^2.0.0" + set-blocking "^2.0.0" + string-width "^3.0.0" + which-module "^2.0.0" + y18n "^4.0.0" + yargs-parser "^13.1.2" + +yargs@^15.3.1, yargs@^15.4.1: + version "15.4.1" + resolved "https://registry.npmmirror.com/yargs/-/yargs-15.4.1.tgz#0d87a16de01aee9d8bec2bfbf74f67851730f4f8" + integrity sha512-aePbxDmcYW++PaqBsJ+HYUFwCdv4LVvdnhBy78E57PIor8/OVvhMrADFFEDh8DHDFRv/O9i3lPhsENjO7QX0+A== + dependencies: + cliui "^6.0.0" + decamelize "^1.2.0" + find-up "^4.1.0" + get-caller-file "^2.0.1" + require-directory "^2.1.1" + require-main-filename "^2.0.0" + set-blocking "^2.0.0" + string-width "^4.2.0" + which-module "^2.0.0" + y18n "^4.0.0" + yargs-parser "^18.1.2" + +yeast@0.1.2: + version "0.1.2" + resolved "https://registry.npmmirror.com/yeast/-/yeast-0.1.2.tgz#008e06d8094320c372dbc2f8ed76a0ca6c8ac419" + integrity sha512-8HFIh676uyGYP6wP13R/j6OJ/1HwJ46snpvzE7aHAN3Ryqh2yX6Xox2B4CUmTwwOIzlG3Bs7ocsP5dZH/R1Qbg== + +yn@3.1.1: + version "3.1.1" + resolved "https://registry.npmmirror.com/yn/-/yn-3.1.1.tgz#1e87401a09d767c1d5eab26a6e4c185182d2eb50" + integrity sha512-Ux4ygGWsu2c7isFWe8Yu1YluJmqVhxqK2cLXNQA5AcC3QfbGNpM7fu0Y8b/z16pXLnFxZYvWhd3fhBY9DLmC6Q== + +yocto-queue@^0.1.0: + version "0.1.0" + resolved "https://registry.npmmirror.com/yocto-queue/-/yocto-queue-0.1.0.tgz#0294eb3dee05028d31ee1a5fa2c556a6aaf10a1b" + integrity sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q== + +yoga-layout-prebuilt@^1.10.0: + version "1.10.0" + resolved "https://registry.npmmirror.com/yoga-layout-prebuilt/-/yoga-layout-prebuilt-1.10.0.tgz#2936fbaf4b3628ee0b3e3b1df44936d6c146faa6" + integrity sha512-YnOmtSbv4MTf7RGJMK0FvZ+KD8OEe/J5BNnR0GHhD8J/XcG/Qvxgszm0Un6FTHWW4uHlTgP0IztiXQnGyIR45g== + dependencies: + "@types/yoga-layout" "1.9.2" + +yurnalist@^2.1.0: + version "2.1.0" + resolved "https://registry.npmmirror.com/yurnalist/-/yurnalist-2.1.0.tgz#44cf7ea5a33a8fab4968cc8c2970489f93760902" + integrity sha512-PgrBqosQLM3gN2xBFIMDLACRTV9c365VqityKKpSTWpwR+U4LAFR3rSVyEoscWlu3EzX9+Y0I86GXUKxpHFl6w== + dependencies: + chalk "^2.4.2" + inquirer "^7.0.0" + is-ci "^2.0.0" + read "^1.0.7" + strip-ansi "^5.2.0" + +zwitch@^1.0.0: + version "1.0.5" + resolved "https://registry.npmmirror.com/zwitch/-/zwitch-1.0.5.tgz#d11d7381ffed16b742f6af7b3f223d5cd9fe9920" + integrity sha512-V50KMwwzqJV0NpZIZFwfOD5/lyny3WlSzRiXgA0G7VUnRlqttta1L6UQIHzd6EuBY/cHGfwTIck7w1yH6Q5zUw==