diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
index 985b1b80ee93..dd67cfe3182f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds;
+import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ServiceException;
import javax.annotation.Nonnull;
@@ -709,21 +710,26 @@ public static int roundupMb(long bytes) {
* or a RpcException.
*/
public static Throwable getUnwrappedException(Exception ex) {
+ Throwable t = ex;
if (ex instanceof ServiceException) {
- Throwable t = ex.getCause();
- if (t instanceof RemoteException) {
- t = ((RemoteException) t).unwrapRemoteException();
- }
- while (t != null) {
- if (t instanceof RpcException ||
- t instanceof AccessControlException ||
- t instanceof SecretManager.InvalidToken) {
- return t;
- }
- t = t.getCause();
+ t = ex.getCause();
+ }
+ if (t instanceof RemoteException) {
+ t = ((RemoteException) t).unwrapRemoteException();
+ }
+ while (t != null) {
+ if (t instanceof RpcException ||
+ t instanceof AccessControlException ||
+ t instanceof SecretManager.InvalidToken) {
+ break;
+ }
+ Throwable cause = t.getCause();
+ if (cause == null || cause instanceof RemoteException) {
+ break;
}
+ t = cause;
}
- return null;
+ return t;
}
/**
@@ -743,7 +749,7 @@ public static boolean shouldNotFailoverOnRpcException(Throwable exception) {
return true;
}
}
- return false;
+ return exception instanceof InvalidProtocolBufferException;
}
/**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
index f3bd1a96b662..cc6695dc7d68 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
@@ -42,10 +42,6 @@ public enum ClientVersion implements ComponentVersion {
"This client version has support for Object Store and File " +
"System Optimized Bucket Layouts."),
- EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST(4,
- "This client version enforces replica index is set for fixing read corruption that could occur when " +
- "replicaIndex parameter is not validated before EC block reads."),
-
FUTURE_VERSION(-1, "Used internally when the server side is older and an"
+ " unknown client version has arrived from the client.");
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 16da798c807e..35be12d46cd1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -115,7 +115,6 @@
import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto.State.RECOVERING;
-import static org.apache.hadoop.ozone.ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST;
import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult;
import org.apache.ratis.statemachine.StateMachine;
@@ -563,15 +562,6 @@ ContainerCommandResponseProto handlePutBlock(
return putBlockResponseSuccess(request, blockDataProto);
}
- /**
- * Checks if a replicaIndex needs to be checked based on the client version for a request.
- * @param request ContainerCommandRequest object.
- * @return true if the validation is required for the client version else false.
- */
- private boolean replicaIndexCheckRequired(ContainerCommandRequestProto request) {
- return request.hasVersion() && request.getVersion() >= EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue();
- }
-
/**
* Handle Get Block operation. Calls BlockManager to process the request.
*/
@@ -590,9 +580,7 @@ ContainerCommandResponseProto handleGetBlock(
try {
BlockID blockID = BlockID.getFromProtobuf(
request.getGetBlock().getBlockID());
- if (replicaIndexCheckRequired(request)) {
- BlockUtils.verifyReplicaIdx(kvContainer, blockID);
- }
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage();
final long numBytes = responseData.getSerializedSize();
metrics.incContainerBytesStats(Type.GetBlock, numBytes);
@@ -709,9 +697,7 @@ ContainerCommandResponseProto handleReadChunk(
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk()
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
- if (replicaIndexCheckRequired(request)) {
- BlockUtils.verifyReplicaIdx(kvContainer, blockID);
- }
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
BlockUtils.verifyBCSId(kvContainer, blockID);
if (dispatcherContext == null) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
index 7773b54f7942..f72ec88516e0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
@@ -248,7 +248,9 @@ public static void verifyBCSId(Container container, BlockID blockID)
public static void verifyReplicaIdx(Container container, BlockID blockID)
throws IOException {
Integer containerReplicaIndex = container.getContainerData().getReplicaIndex();
- if (containerReplicaIndex > 0 && !containerReplicaIndex.equals(blockID.getReplicaIndex())) {
+ Integer blockReplicaIndex = blockID.getReplicaIndex();
+ if (containerReplicaIndex > 0 && blockReplicaIndex != null && blockReplicaIndex != 0 &&
+ !containerReplicaIndex.equals(blockReplicaIndex)) {
throw new StorageContainerException(
"Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container "
+ container.getContainerData().getContainerID() + " replicaIdx is "
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
index 7fd4cf3fcb8c..3e10ab765279 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
@@ -115,8 +115,7 @@ public void testGetBlockWithReplicaIndexMismatch(ClientVersion clientVersion, in
handler.handleGetBlock(
getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.GetBlock, rid),
container);
- assertEquals((replicaIndex > 0 && rid != replicaIndex && clientVersion.toProtoValue() >=
- ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ?
+ assertEquals((replicaIndex > 0 && rid != 0 && rid != replicaIndex) ?
ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID,
response.getResult());
}
@@ -158,8 +157,7 @@ public void testReadChunkWithReplicaIndexMismatch(ClientVersion clientVersion, i
ContainerProtos.ContainerCommandResponseProto response =
handler.handleReadChunk(getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.ReadChunk, rid),
container, null);
- assertEquals((replicaIndex > 0 && rid != replicaIndex &&
- clientVersion.toProtoValue() >= ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ?
+ assertEquals((replicaIndex > 0 && rid != 0 && rid != replicaIndex) ?
ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID,
response.getResult());
}
diff --git a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/TestMiniChaosOzoneCluster.java b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/TestMiniChaosOzoneCluster.java
index ab62b920b714..6894aed25ab6 100644
--- a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/TestMiniChaosOzoneCluster.java
+++ b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/TestMiniChaosOzoneCluster.java
@@ -30,9 +30,9 @@
import org.apache.hadoop.ozone.freon.FreonReplicationOptions;
import org.apache.hadoop.ozone.loadgenerators.LoadGenerator;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Test;
import picocli.CommandLine;
import picocli.CommandLine.Command;
import picocli.CommandLine.Option;
@@ -117,7 +117,7 @@ enum AllowedBucketLayouts { FILE_SYSTEM_OPTIMIZED, OBJECT_STORE }
private static final String OM_SERVICE_ID = "ozoneChaosTest";
private static final String SCM_SERVICE_ID = "scmChaosTest";
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
OzoneConfiguration configuration = new OzoneConfiguration();
@@ -191,7 +191,7 @@ static void setNumManagers(int nOms, int numScms, boolean enableHA) {
/**
* Shutdown MiniDFSCluster.
*/
- @AfterClass
+ @AfterAll
public static void shutdown() {
if (loadGenerator != null) {
loadGenerator.shutdownLoadGenerator();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
index 4f7ae6d410a2..b7cff225d7f0 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
@@ -56,33 +56,26 @@
import static java.util.Collections.singletonList;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
import org.apache.ratis.protocol.exceptions.NotReplicatedException;
import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
import org.apache.ratis.protocol.exceptions.TimeoutIOException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
/**
* Class to test CommitWatcher functionality.
*/
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestCommitWatcher {
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
private MiniOzoneCluster cluster;
private OzoneConfiguration conf = new OzoneConfiguration();
private OzoneClient client;
@@ -104,7 +97,7 @@ public class TestCommitWatcher {
*
* @throws IOException
*/
- @Before
+ @BeforeEach
public void init() throws Exception {
chunkSize = (int)(1 * OzoneConsts.MB);
flushSize = (long) 2 * chunkSize;
@@ -165,7 +158,7 @@ public void init() throws Exception {
/**
* Shutdown MiniDFSCluster.
*/
- @After
+ @AfterEach
public void shutdown() {
IOUtils.closeQuietly(client);
if (cluster != null) {
@@ -324,11 +317,12 @@ public void testReleaseBuffersOnException() throws Exception {
// can itself get AlreadyClosedException from the Ratis Server
// and the write may fail with RaftRetryFailureException
Throwable t = HddsClientUtils.checkForException(ioe);
- assertTrue("Unexpected exception: " + t.getClass(),
+ assertTrue(
t instanceof RaftRetryFailureException ||
t instanceof TimeoutIOException ||
t instanceof AlreadyClosedException ||
- t instanceof NotReplicatedException);
+ t instanceof NotReplicatedException,
+ "Unexpected exception: " + t.getClass());
}
if (ratisClient.getReplicatedMinCommitIndex() < replies.get(1)
.getLogIndex()) {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java
index a0eafc4fdf36..9283a9482d72 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java
@@ -460,7 +460,7 @@ public void testCreateRecoveryContainer() throws Exception {
int replicaIndex = 4;
XceiverClientSpi dnClient = xceiverClientManager.acquireClient(
createSingleNodePipeline(newPipeline, newPipeline.getNodes().get(0),
- replicaIndex));
+ 2));
try {
// To create the actual situation, container would have been in closed
// state at SCM.
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java
index 752c011f4688..b99a58821e3b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java
@@ -48,13 +48,10 @@
import org.apache.hadoop.security.token.Token;
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ratis.util.ExitUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -65,6 +62,7 @@
import java.util.Objects;
import java.util.Properties;
import java.util.UUID;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
@@ -93,14 +91,15 @@
import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
import static org.apache.ozone.test.GenericTestUtils.assertExceptionContains;
import static org.apache.ozone.test.GenericTestUtils.waitFor;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThrows;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Integration test to verify block tokens in a secure cluster.
*/
@InterfaceAudience.Private
+@Timeout(value = 180, unit = TimeUnit.SECONDS)
public final class TestBlockTokens {
private static final Logger LOG = LoggerFactory
.getLogger(TestBlockTokens.class);
@@ -111,9 +110,6 @@ public final class TestBlockTokens {
private static final int EXPIRY_DURATION_IN_MS = 10000;
private static final int ROTATION_CHECK_DURATION_IN_MS = 100;
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(180));
-
private static MiniKdc miniKdc;
private static OzoneConfiguration conf;
private static File workDir;
@@ -129,7 +125,7 @@ public final class TestBlockTokens {
private static BlockInputStreamFactory blockInputStreamFactory =
new BlockInputStreamFactoryImpl();
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
conf = new OzoneConfiguration();
conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "localhost");
@@ -161,7 +157,7 @@ private static void createTestData() throws IOException {
}
}
- @AfterClass
+ @AfterAll
public static void stop() {
miniKdc.stop();
IOUtils.close(LOG, client);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokensCLI.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokensCLI.java
index 06b6b2d96dc1..9a98a0a1897e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokensCLI.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokensCLI.java
@@ -33,14 +33,6 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ratis.util.ExitUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.jupiter.api.Assertions;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,6 +48,7 @@
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static java.time.Duration.between;
@@ -78,19 +71,20 @@
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
import static org.junit.jupiter.api.Assertions.assertEquals;
-
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.Assertions;
/**
* Integration test class to verify block token CLI commands functionality in a
* secure cluster.
*/
@InterfaceAudience.Private
+@Timeout(value = 180, unit = TimeUnit.SECONDS)
public final class TestBlockTokensCLI {
private static final Logger LOG = LoggerFactory
.getLogger(TestBlockTokensCLI.class);
-
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(180));
-
private static MiniKdc miniKdc;
private static OzoneAdmin ozoneAdmin;
private static OzoneConfiguration conf;
@@ -105,7 +99,7 @@ public final class TestBlockTokensCLI {
private static MiniOzoneHAClusterImpl cluster;
private static OzoneClient client;
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
conf = new OzoneConfiguration();
conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "localhost");
@@ -128,7 +122,7 @@ public static void init() throws Exception {
ozoneAdmin = new OzoneAdmin(conf);
}
- @AfterClass
+ @AfterAll
public static void stop() {
miniKdc.stop();
IOUtils.close(LOG, client);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
index 8a35f87e173d..2334e3cf94f1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
@@ -26,37 +26,30 @@
import org.apache.ozone.test.UnhealthyTest;
import org.apache.ozone.test.tag.Unhealthy;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
import org.junit.experimental.categories.Category;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import org.junit.jupiter.api.Timeout;
import java.util.Optional;
+import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* This class tests container balancer operations
* from cblock clients.
*/
+@Timeout(value = 300, unit = TimeUnit.MILLISECONDS)
public class TestContainerBalancerOperations {
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
-
private static ScmClient containerBalancerClient;
private static MiniOzoneCluster cluster;
private static OzoneConfiguration ozoneConf;
- @BeforeClass
+ @BeforeAll
public static void setup() throws Exception {
ozoneConf = new OzoneConfiguration();
ozoneConf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
@@ -66,7 +59,7 @@ public static void setup() throws Exception {
cluster.waitForClusterToBeReady();
}
- @AfterClass
+ @AfterAll
public static void cleanup() throws Exception {
if (cluster != null) {
cluster.shutdown();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
index f5950dde84ed..56e744886b82 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
@@ -1,4 +1,4 @@
-/**
+ /**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -29,39 +29,32 @@
import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Rule;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
import java.io.IOException;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.REPLICATION;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.Assertions;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* This class tests container operations (TODO currently only supports create)
* from cblock clients.
*/
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestContainerOperations {
-
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
-
private static ScmClient storageClient;
private static MiniOzoneCluster cluster;
private static OzoneConfiguration ozoneConf;
- @BeforeClass
+ @BeforeAll
public static void setup() throws Exception {
ozoneConf = new OzoneConfiguration();
ozoneConf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
@@ -71,7 +64,7 @@ public static void setup() throws Exception {
cluster.waitForClusterToBeReady();
}
- @AfterClass
+ @AfterAll
public static void cleanup() throws Exception {
if (cluster != null) {
cluster.shutdown();
@@ -100,13 +93,13 @@ public void testCreate() throws Exception {
public void testGetPipeline() throws Exception {
try {
storageClient.getPipeline(PipelineID.randomId().getProtobuf());
- Assert.fail("Get Pipeline should fail");
+ Assertions.fail("Get Pipeline should fail");
} catch (Exception e) {
assertTrue(
SCMHAUtils.unwrapException(e) instanceof PipelineNotFoundException);
}
- Assert.assertFalse(storageClient.listPipelines().isEmpty());
+ Assertions.assertFalse(storageClient.listPipelines().isEmpty());
}
@Test
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java
index bdf41afd4180..6e76a86dd947 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java
@@ -37,37 +37,28 @@
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.Assertions;
import java.io.IOException;
-
-import org.junit.Rule;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import java.util.concurrent.TimeUnit;
import static java.nio.charset.StandardCharsets.UTF_8;
/**
* Tests the idempotent operations in ContainerStateMachine.
*/
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestContainerStateMachineIdempotency {
-
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
private static MiniOzoneCluster cluster;
private static OzoneConfiguration ozoneConfig;
private static StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient;
private static XceiverClientManager xceiverClientManager;
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
ozoneConfig = new OzoneConfiguration();
ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
@@ -80,7 +71,7 @@ public static void init() throws Exception {
xceiverClientManager = new XceiverClientManager(ozoneConfig);
}
- @AfterClass
+ @AfterAll
public static void shutdown() {
if (cluster != null) {
cluster.shutdown();
@@ -124,7 +115,7 @@ public void testContainerStateMachineIdempotency() throws Exception {
ContainerProtocolCalls.closeContainer(client, containerID, null);
ContainerProtocolCalls.closeContainer(client, containerID, null);
} catch (IOException ioe) {
- Assert.fail("Container operation failed" + ioe);
+ Assertions.fail("Container operation failed" + ioe);
}
xceiverClientManager.releaseClient(client, false);
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneOMHACluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneOMHACluster.java
index 87cd2f831de2..a3e0be5a85d3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneOMHACluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneOMHACluster.java
@@ -21,27 +21,25 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.ozone.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
-import java.io.IOException;
import java.util.UUID;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.Assertions;
+
/**
* This class tests MiniOzoneHAClusterImpl.
*/
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestMiniOzoneOMHACluster {
private MiniOzoneHAClusterImpl cluster = null;
@@ -51,18 +49,12 @@ public class TestMiniOzoneOMHACluster {
private String omServiceId;
private int numOfOMs = 3;
- @Rule
- public ExpectedException exception = ExpectedException.none();
-
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
-
/**
* Create a MiniOzoneHAClusterImpl for testing.
*
* @throws IOException
*/
- @Before
+ @BeforeEach
public void init() throws Exception {
conf = new OzoneConfiguration();
clusterId = UUID.randomUUID().toString();
@@ -83,7 +75,7 @@ public void init() throws Exception {
/**
* Shutdown MiniOzoneHAClusterImpl.
*/
- @After
+ @AfterEach
public void shutdown() {
if (cluster != null) {
cluster.shutdown();
@@ -99,9 +91,9 @@ public void testGetOMLeader() throws InterruptedException, TimeoutException {
ozoneManager.set(om);
return om != null;
}, 100, 120000);
- Assert.assertNotNull("Timed out waiting OM leader election to finish: "
- + "no leader or more than one leader.", ozoneManager);
- Assert.assertTrue("Should have gotten the leader!",
- ozoneManager.get().isLeaderReady());
+ Assertions.assertNotNull(ozoneManager, "Timed out waiting OM leader election to finish: "
+ + "no leader or more than one leader.");
+ Assertions.assertTrue(ozoneManager.get().isLeaderReady(),
+ "Should have gotten the leader!");
}
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java
index 1351d028b724..f35506eccd5d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java
@@ -21,14 +21,12 @@
import java.io.StringWriter;
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
import picocli.CommandLine;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -49,28 +47,26 @@
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
-import static org.junit.Assert.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Test cases for LeaseRecoverer.
*/
+@Timeout(value = 120000, unit = TimeUnit.MILLISECONDS)
public class TestLeaseRecoverer {
private static MiniOzoneCluster cluster = null;
private static OzoneConfiguration conf;
private static OzoneBucket fsoOzoneBucket;
private static OzoneClient client;
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(new Timeout(120000));
-
/**
* Create a MiniDFSCluster for testing.
*
*
* @throws IOException
*/
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
conf = new OzoneConfiguration();
conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
@@ -88,7 +84,7 @@ public static void init() throws Exception {
.createVolumeAndBucket(client, BucketLayout.FILE_SYSTEM_OPTIMIZED);
}
- @AfterClass
+ @AfterAll
public static void teardownClass() {
IOUtils.closeQuietly(client);
if (cluster != null) {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java
index aa181e91bbb6..77815698c3b8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java
@@ -37,9 +37,11 @@
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ratis.util.FileUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -48,28 +50,17 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
-
-import org.junit.Rule;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Test cases for ContainerMapper.
*/
-
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestContainerMapper {
-
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
-
private static MiniOzoneCluster cluster = null;
private static OzoneClient ozClient = null;
private static ObjectStore store = null;
@@ -84,7 +75,7 @@ public class TestContainerMapper {
private static String dbPath;
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
conf = new OzoneConfiguration();
dbPath = GenericTestUtils.getRandomizedTempPath();
@@ -144,7 +135,7 @@ private static byte[] generateData(int size, byte val) {
return chars;
}
- @AfterClass
+ @AfterAll
public static void shutdown() throws IOException {
IOUtils.closeQuietly(ozClient);
cluster.shutdown();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerReportWithKeys.java
index bdbe4107fbe1..223936fc8f96 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerReportWithKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerReportWithKeys.java
@@ -37,34 +37,25 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.Assertions;
import java.io.IOException;
import java.util.HashMap;
import java.util.Set;
+import java.util.concurrent.TimeUnit;
import static java.nio.charset.StandardCharsets.UTF_8;
/**
* This class tests container report with DN container state info.
*/
+@Timeout(value = 300, unit = TimeUnit.SECONDS)
public class TestContainerReportWithKeys {
-
- /**
- * Set a timeout for each test.
- */
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(300));
private static final Logger LOG = LoggerFactory.getLogger(
TestContainerReportWithKeys.class);
private static MiniOzoneCluster cluster = null;
@@ -72,9 +63,6 @@ public class TestContainerReportWithKeys {
private static OzoneConfiguration conf;
private static StorageContainerManager scm;
- @Rule
- public ExpectedException exception = ExpectedException.none();
-
/**
* Create a MiniDFSCluster for testing.
*
@@ -82,7 +70,7 @@ public class TestContainerReportWithKeys {
*
* @throws IOException
*/
- @BeforeClass
+ @BeforeAll
public static void init() throws Exception {
conf = new OzoneConfiguration();
cluster = MiniOzoneCluster.newBuilder(conf).build();
@@ -94,7 +82,7 @@ public static void init() throws Exception {
/**
* Shutdown MiniDFSCluster.
*/
- @AfterClass
+ @AfterAll
public static void shutdown() {
IOUtils.closeQuietly(client);
if (cluster != null) {
@@ -140,9 +128,9 @@ public void testContainerReportKeyWrite() throws Exception {
Set replicas =
scm.getContainerManager().getContainerReplicas(
ContainerID.valueOf(keyInfo.getContainerID()));
- Assert.assertTrue(replicas.size() == 1);
+ Assertions.assertTrue(replicas.size() == 1);
replicas.stream().forEach(rp ->
- Assert.assertTrue(rp.getDatanodeDetails().getParent() != null));
+ Assertions.assertTrue(rp.getDatanodeDetails().getParent() != null));
LOG.info("SCM Container Info keyCount: {} usedBytes: {}",
cinfo.getNumberOfKeys(), cinfo.getUsedBytes());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java
index d958ca361152..065d367382aa 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java
@@ -37,13 +37,10 @@
import org.apache.ozone.test.tag.Flaky;
import org.apache.ratis.util.ExitUtils;
import org.jetbrains.annotations.NotNull;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.apache.ozone.test.JUnit5AwareTimeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -53,6 +50,7 @@
import java.util.List;
import java.util.Properties;
import java.util.UUID;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
@@ -77,23 +75,21 @@
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThrows;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Integration test to verify symmetric SecretKeys APIs in a secure cluster.
*/
+
+@Timeout(value = 500, unit = TimeUnit.SECONDS)
@InterfaceAudience.Private
public final class TestSecretKeysApi {
private static final Logger LOG = LoggerFactory
.getLogger(TestSecretKeysApi.class);
-
- @Rule
- public TestRule timeout = new JUnit5AwareTimeout(Timeout.seconds(500));
-
private MiniKdc miniKdc;
private OzoneConfiguration conf;
private File workDir;
@@ -106,7 +102,7 @@ public final class TestSecretKeysApi {
private String scmId;
private MiniOzoneHAClusterImpl cluster;
- @Before
+ @BeforeEach
public void init() throws Exception {
conf = new OzoneConfiguration();
conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, "localhost");
@@ -123,7 +119,7 @@ public void init() throws Exception {
createCredentialsInKDC();
}
- @After
+ @AfterEach
public void stop() {
miniKdc.stop();
if (cluster != null) {
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzFs.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzFs.java
index acab6d168cc4..9f7551aa8f03 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzFs.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzFs.java
@@ -46,4 +46,13 @@ public BasicOzFs(URI theUri, Configuration conf)
public int getUriDefaultPort() {
return -1;
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
index b5012f95c4ec..e1cb391da539 100644
--- a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
+++ b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
@@ -45,4 +45,13 @@ public OzFs(URI theUri, Configuration conf)
public int getUriDefaultPort() {
return -1;
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
index 4cd04da9c867..0f421a855239 100644
--- a/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
+++ b/hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
@@ -46,4 +46,13 @@ public RootedOzFs(URI theUri, Configuration conf)
public int getUriDefaultPort() {
return -1;
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
index 914832e2cfab..b1e046547fa0 100644
--- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
+++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
@@ -40,4 +40,13 @@ public OzFs(URI theUri, Configuration conf)
super(theUri, new OzoneFileSystem(), conf,
OzoneConsts.OZONE_URI_SCHEME, false);
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
index 076287eaac14..81bbaacd7c87 100644
--- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
+++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
@@ -41,4 +41,13 @@ public RootedOzFs(URI theUri, Configuration conf)
super(theUri, new RootedOzoneFileSystem(), conf,
OzoneConsts.OZONE_OFS_URI_SCHEME, false);
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
index 346b994a3aec..548e11f5d484 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
@@ -41,4 +41,13 @@ public OzFs(URI theUri, Configuration conf)
super(theUri, new OzoneFileSystem(), conf,
OzoneConsts.OZONE_URI_SCHEME, false);
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
index 076287eaac14..81bbaacd7c87 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzFs.java
@@ -41,4 +41,13 @@ public RootedOzFs(URI theUri, Configuration conf)
super(theUri, new RootedOzoneFileSystem(), conf,
OzoneConsts.OZONE_OFS_URI_SCHEME, false);
}
+
+ /**
+ * Close the file system; the FileContext API doesn't have an explicit close.
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ fsImpl.close();
+ super.finalize();
+ }
}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
index f5c7d5afde44..d71f85c065f0 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json
@@ -62,7 +62,7 @@
"@types/react-select": "^3.0.13",
"@typescript-eslint/eslint-plugin": "^5.30.0",
"@typescript-eslint/parser": "^5.30.0",
- "@vitejs/plugin-react": "^4.0.0",
+ "@vitejs/plugin-react-swc": "^3.5.0",
"eslint": "^7.28.0",
"eslint-config-prettier": "^8.10.0",
"eslint-plugin-prettier": "^3.4.1",
@@ -70,7 +70,6 @@
"npm-run-all": "^4.1.5",
"prettier": "^2.8.4",
"vite": "4.5.3",
- "vite-plugin-svgr": "^4.2.0",
"vite-tsconfig-paths": "^3.6.0",
"vitest": "^1.6.0"
},
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml
index 790f4ed7f5e8..011b61a2110f 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml
@@ -79,9 +79,9 @@ devDependencies:
'@typescript-eslint/parser':
specifier: ^5.30.0
version: 5.62.0(eslint@7.32.0)(typescript@4.9.5)
- '@vitejs/plugin-react':
- specifier: ^4.0.0
- version: 4.3.1(vite@4.5.3)
+ '@vitejs/plugin-react-swc':
+ specifier: ^3.5.0
+ version: 3.7.1(vite@4.5.3)
eslint:
specifier: ^7.28.0
version: 7.32.0
@@ -103,9 +103,6 @@ devDependencies:
vite:
specifier: 4.5.3
version: 4.5.3(less@3.13.1)
- vite-plugin-svgr:
- specifier: ^4.2.0
- version: 4.2.0(typescript@4.9.5)(vite@4.5.3)
vite-tsconfig-paths:
specifier: ^3.6.0
version: 3.6.0(vite@4.5.3)
@@ -115,14 +112,6 @@ devDependencies:
packages:
- /@ampproject/remapping@2.3.0:
- resolution: {integrity: sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==}
- engines: {node: '>=6.0.0'}
- dependencies:
- '@jridgewell/gen-mapping': 0.3.5
- '@jridgewell/trace-mapping': 0.3.25
- dev: true
-
/@ant-design/colors@5.1.1:
resolution: {integrity: sha512-Txy4KpHrp3q4XZdfgOBqLl+lkQIc3tEvHXOimRN1giX1AEC7mGtyrO9p8iRGJ3FLuVMGa2gNEzQyghVymLttKQ==}
dependencies:
@@ -148,7 +137,7 @@ packages:
dependencies:
'@ant-design/colors': 6.0.0
'@ant-design/icons-svg': 4.4.2
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
lodash: 4.17.21
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -161,7 +150,7 @@ packages:
peerDependencies:
react: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
json2mq: 0.2.0
lodash: 4.17.21
@@ -172,224 +161,103 @@ packages:
/@babel/code-frame@7.12.11:
resolution: {integrity: sha512-Zt1yodBx1UcyiePMSkWnU4hPqhwq7hGi2nFL1LeA3EUl+q2LQx16MISgJ0+z7dnmgvP9QtIleuETGOiOH1RcIw==}
dependencies:
- '@babel/highlight': 7.24.7
+ '@babel/highlight': 7.25.9
dev: true
- /@babel/code-frame@7.24.7:
- resolution: {integrity: sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==}
+ /@babel/code-frame@7.26.0:
+ resolution: {integrity: sha512-INCKxTtbXtcNbUZ3YXutwMpEleqttcswhAdee7dhuoVrD2cnuc3PqtERBtxkX5nziX9vnBL8WXmSGwv8CuPV6g==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/highlight': 7.24.7
- picocolors: 1.0.1
-
- /@babel/compat-data@7.24.8:
- resolution: {integrity: sha512-c4IM7OTg6k1Q+AJ153e2mc2QVTezTwnb4VzquwcyiEzGnW0Kedv4do/TrkU98qPeC5LNiMt/QXwIjzYXLBpyZg==}
- engines: {node: '>=6.9.0'}
- dev: true
-
- /@babel/core@7.24.8:
- resolution: {integrity: sha512-6AWcmZC/MZCO0yKys4uhg5NlxL0ESF3K6IAaoQ+xSXvPyPyxNWRafP+GDbI88Oh68O7QkJgmEtedWPM9U0pZNg==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@ampproject/remapping': 2.3.0
- '@babel/code-frame': 7.24.7
- '@babel/generator': 7.24.8
- '@babel/helper-compilation-targets': 7.24.8
- '@babel/helper-module-transforms': 7.24.8(@babel/core@7.24.8)
- '@babel/helpers': 7.24.8
- '@babel/parser': 7.24.8
- '@babel/template': 7.24.7
- '@babel/traverse': 7.24.8
- '@babel/types': 7.24.8
- convert-source-map: 2.0.0
- debug: 4.3.5
- gensync: 1.0.0-beta.2
- json5: 2.2.3
- semver: 6.3.1
- transitivePeerDependencies:
- - supports-color
- dev: true
+ '@babel/helper-validator-identifier': 7.25.9
+ js-tokens: 4.0.0
+ picocolors: 1.1.1
+ dev: false
- /@babel/generator@7.24.8:
- resolution: {integrity: sha512-47DG+6F5SzOi0uEvK4wMShmn5yY0mVjVJoWTphdY2B4Rx9wHgjK7Yhtr0ru6nE+sn0v38mzrWOlah0p/YlHHOQ==}
+ /@babel/generator@7.26.0:
+ resolution: {integrity: sha512-/AIkAmInnWwgEAJGQr9vY0c66Mj6kjkE2ZPB1PurTRaRAh3U+J45sAQMjQDJdh4WbR3l0x5xkimXBKyBXXAu2w==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/types': 7.24.8
+ '@babel/parser': 7.26.1
+ '@babel/types': 7.26.0
'@jridgewell/gen-mapping': 0.3.5
'@jridgewell/trace-mapping': 0.3.25
- jsesc: 2.5.2
-
- /@babel/helper-compilation-targets@7.24.8:
- resolution: {integrity: sha512-oU+UoqCHdp+nWVDkpldqIQL/i/bvAv53tRqLG/s+cOXxe66zOYLU7ar/Xs3LdmBihrUMEUhwu6dMZwbNOYDwvw==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/compat-data': 7.24.8
- '@babel/helper-validator-option': 7.24.8
- browserslist: 4.23.2
- lru-cache: 5.1.1
- semver: 6.3.1
- dev: true
-
- /@babel/helper-environment-visitor@7.24.7:
- resolution: {integrity: sha512-DoiN84+4Gnd0ncbBOM9AZENV4a5ZiL39HYMyZJGZ/AZEykHYdJw0wW3kdcsh9/Kn+BRXHLkkklZ51ecPKmI1CQ==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/types': 7.24.8
-
- /@babel/helper-function-name@7.24.7:
- resolution: {integrity: sha512-FyoJTsj/PEUWu1/TYRiXTIHc8lbw+TDYkZuoE43opPS5TrI7MyONBE1oNvfguEXAD9yhQRrVBnXdXzSLQl9XnA==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/template': 7.24.7
- '@babel/types': 7.24.8
-
- /@babel/helper-hoist-variables@7.24.7:
- resolution: {integrity: sha512-MJJwhkoGy5c4ehfoRyrJ/owKeMl19U54h27YYftT0o2teQ3FJ3nQUf/I3LlJsX4l3qlw7WRXUmiyajvHXoTubQ==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/types': 7.24.8
-
- /@babel/helper-module-imports@7.24.7:
- resolution: {integrity: sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/traverse': 7.24.8
- '@babel/types': 7.24.8
- transitivePeerDependencies:
- - supports-color
-
- /@babel/helper-module-transforms@7.24.8(@babel/core@7.24.8):
- resolution: {integrity: sha512-m4vWKVqvkVAWLXfHCCfff2luJj86U+J0/x+0N3ArG/tP0Fq7zky2dYwMbtPmkc/oulkkbjdL3uWzuoBwQ8R00Q==}
- engines: {node: '>=6.9.0'}
- peerDependencies:
- '@babel/core': ^7.0.0
- dependencies:
- '@babel/core': 7.24.8
- '@babel/helper-environment-visitor': 7.24.7
- '@babel/helper-module-imports': 7.24.7
- '@babel/helper-simple-access': 7.24.7
- '@babel/helper-split-export-declaration': 7.24.7
- '@babel/helper-validator-identifier': 7.24.7
- transitivePeerDependencies:
- - supports-color
- dev: true
-
- /@babel/helper-plugin-utils@7.24.8:
- resolution: {integrity: sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==}
- engines: {node: '>=6.9.0'}
- dev: true
+ jsesc: 3.0.2
+ dev: false
- /@babel/helper-simple-access@7.24.7:
- resolution: {integrity: sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==}
+ /@babel/helper-module-imports@7.25.9:
+ resolution: {integrity: sha512-tnUA4RsrmflIM6W6RFTLFSXITtl0wKjgpnLgXyowocVPrbYrLUXSBXDgTs8BlbmIzIdlBySRQjINYs2BAkiLtw==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/traverse': 7.24.8
- '@babel/types': 7.24.8
+ '@babel/traverse': 7.25.9
+ '@babel/types': 7.26.0
transitivePeerDependencies:
- supports-color
- dev: true
-
- /@babel/helper-split-export-declaration@7.24.7:
- resolution: {integrity: sha512-oy5V7pD+UvfkEATUKvIjvIAH/xCzfsFVw7ygW2SI6NClZzquT+mwdTfgfdbUiceh6iQO0CHtCPsyze/MZ2YbAA==}
- engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/types': 7.24.8
-
- /@babel/helper-string-parser@7.24.8:
- resolution: {integrity: sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==}
- engines: {node: '>=6.9.0'}
-
- /@babel/helper-validator-identifier@7.24.7:
- resolution: {integrity: sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==}
- engines: {node: '>=6.9.0'}
+ dev: false
- /@babel/helper-validator-option@7.24.8:
- resolution: {integrity: sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==}
+ /@babel/helper-string-parser@7.25.9:
+ resolution: {integrity: sha512-4A/SCr/2KLd5jrtOMFzaKjVtAei3+2r/NChoBNoZ3EyP/+GlhoaEGoWOZUmFmoITP7zOJyHIMm+DYRd8o3PvHA==}
engines: {node: '>=6.9.0'}
- dev: true
+ dev: false
- /@babel/helpers@7.24.8:
- resolution: {integrity: sha512-gV2265Nkcz7weJJfvDoAEVzC1e2OTDpkGbEsebse8koXUJUXPsCMi7sRo/+SPMuMZ9MtUPnGwITTnQnU5YjyaQ==}
+ /@babel/helper-validator-identifier@7.25.9:
+ resolution: {integrity: sha512-Ed61U6XJc3CVRfkERJWDz4dJwKe7iLmmJsbOGu9wSloNSFttHV0I8g6UAgb7qnK5ly5bGLPd4oXZlxCdANBOWQ==}
engines: {node: '>=6.9.0'}
- dependencies:
- '@babel/template': 7.24.7
- '@babel/types': 7.24.8
- dev: true
- /@babel/highlight@7.24.7:
- resolution: {integrity: sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==}
+ /@babel/highlight@7.25.9:
+ resolution: {integrity: sha512-llL88JShoCsth8fF8R4SJnIn+WLvR6ccFxu1H3FlMhDontdcmZWf2HgIZ7AIqV3Xcck1idlohrN4EUBQz6klbw==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/helper-validator-identifier': 7.24.7
+ '@babel/helper-validator-identifier': 7.25.9
chalk: 2.4.2
js-tokens: 4.0.0
- picocolors: 1.0.1
+ picocolors: 1.1.1
+ dev: true
- /@babel/parser@7.24.8:
- resolution: {integrity: sha512-WzfbgXOkGzZiXXCqk43kKwZjzwx4oulxZi3nq2TYL9mOjQv6kYwul9mz6ID36njuL7Xkp6nJEfok848Zj10j/w==}
+ /@babel/parser@7.26.1:
+ resolution: {integrity: sha512-reoQYNiAJreZNsJzyrDNzFQ+IQ5JFiIzAHJg9bn94S3l+4++J7RsIhNMoB+lgP/9tpmiAQqspv+xfdxTSzREOw==}
engines: {node: '>=6.0.0'}
hasBin: true
dependencies:
- '@babel/types': 7.24.8
-
- /@babel/plugin-transform-react-jsx-self@7.24.7(@babel/core@7.24.8):
- resolution: {integrity: sha512-fOPQYbGSgH0HUp4UJO4sMBFjY6DuWq+2i8rixyUMb3CdGixs/gccURvYOAhajBdKDoGajFr3mUq5rH3phtkGzw==}
- engines: {node: '>=6.9.0'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
- '@babel/helper-plugin-utils': 7.24.8
- dev: true
-
- /@babel/plugin-transform-react-jsx-source@7.24.7(@babel/core@7.24.8):
- resolution: {integrity: sha512-J2z+MWzZHVOemyLweMqngXrgGC42jQ//R0KdxqkIz/OrbVIIlhFI3WigZ5fO+nwFvBlncr4MGapd8vTyc7RPNQ==}
- engines: {node: '>=6.9.0'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
- '@babel/helper-plugin-utils': 7.24.8
- dev: true
+ '@babel/types': 7.26.0
+ dev: false
- /@babel/runtime@7.24.8:
- resolution: {integrity: sha512-5F7SDGs1T72ZczbRwbGO9lQi0NLjQxzl6i4lJxLxfW9U5UluCSyEJeniWvnhl3/euNiqQVbo8zruhsDfid0esA==}
+ /@babel/runtime@7.26.0:
+ resolution: {integrity: sha512-FDSOghenHTiToteC/QRlv2q3DhPZ/oOXTBoirfWNx1Cx3TMVcGWQtMMmQcSvb/JjpNeGzx8Pq/b4fKEJuWm1sw==}
engines: {node: '>=6.9.0'}
dependencies:
regenerator-runtime: 0.14.1
- /@babel/template@7.24.7:
- resolution: {integrity: sha512-jYqfPrU9JTF0PmPy1tLYHW4Mp4KlgxJD9l2nP9fD6yT/ICi554DmrWBAEYpIelzjHf1msDP3PxJIRt/nFNfBig==}
+ /@babel/template@7.25.9:
+ resolution: {integrity: sha512-9DGttpmPvIxBb/2uwpVo3dqJ+O6RooAFOS+lB+xDqoE2PVCE8nfoHMdZLpfCQRLwvohzXISPZcgxt80xLfsuwg==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/code-frame': 7.24.7
- '@babel/parser': 7.24.8
- '@babel/types': 7.24.8
+ '@babel/code-frame': 7.26.0
+ '@babel/parser': 7.26.1
+ '@babel/types': 7.26.0
+ dev: false
- /@babel/traverse@7.24.8:
- resolution: {integrity: sha512-t0P1xxAPzEDcEPmjprAQq19NWum4K0EQPjMwZQZbHt+GiZqvjCHjj755Weq1YRPVzBI+3zSfvScfpnuIecVFJQ==}
+ /@babel/traverse@7.25.9:
+ resolution: {integrity: sha512-ZCuvfwOwlz/bawvAuvcj8rrithP2/N55Tzz342AkTvq4qaWbGfmCk/tKhNaV2cthijKrPAA8SRJV5WWe7IBMJw==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/code-frame': 7.24.7
- '@babel/generator': 7.24.8
- '@babel/helper-environment-visitor': 7.24.7
- '@babel/helper-function-name': 7.24.7
- '@babel/helper-hoist-variables': 7.24.7
- '@babel/helper-split-export-declaration': 7.24.7
- '@babel/parser': 7.24.8
- '@babel/types': 7.24.8
- debug: 4.3.5
+ '@babel/code-frame': 7.26.0
+ '@babel/generator': 7.26.0
+ '@babel/parser': 7.26.1
+ '@babel/template': 7.25.9
+ '@babel/types': 7.26.0
+ debug: 4.3.7
globals: 11.12.0
transitivePeerDependencies:
- supports-color
+ dev: false
- /@babel/types@7.24.8:
- resolution: {integrity: sha512-SkSBEHwwJRU52QEVZBmMBnE5Ux2/6WU1grdYyOhpbCNxbmJrDuDCphBzKZSO3taf0zztp+qkWlymE5tVL5l0TA==}
+ /@babel/types@7.26.0:
+ resolution: {integrity: sha512-Z/yiTPj+lDVnF7lWeKCIJzaIkI0vYO87dMpZ4bg4TDrFe4XXLFWL1TbXU27gBP3QccxV9mZICCrnjnYlJjXHOA==}
engines: {node: '>=6.9.0'}
dependencies:
- '@babel/helper-string-parser': 7.24.8
- '@babel/helper-validator-identifier': 7.24.7
- to-fast-properties: 2.0.0
+ '@babel/helper-string-parser': 7.25.9
+ '@babel/helper-validator-identifier': 7.25.9
+ dev: false
/@ctrl/tinycolor@3.6.1:
resolution: {integrity: sha512-SITSV6aIXsuVNV3f3O0f2n/cgyEDWoSqtZMYiAmcsYHydcKrOz3gUxB/iXd/Qf08+IZX4KpgNbvUdMBmWz+kcA==}
@@ -414,7 +282,7 @@ packages:
peerDependencies:
react: '>=16.3.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
'@emotion/cache': 10.0.29
'@emotion/css': 10.0.27
'@emotion/serialize': 0.11.16
@@ -878,8 +746,8 @@ packages:
dev: true
optional: true
- /@eslint-community/eslint-utils@4.4.0(eslint@7.32.0):
- resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==}
+ /@eslint-community/eslint-utils@4.4.1(eslint@7.32.0):
+ resolution: {integrity: sha512-s3O3waFUrMV8P/XaF/+ZTp1X9XBZW1a4B97ZnjQF2KYWaFD2A8KyFBsrsfSjEmjn3RGWAIuvlneuZm3CUK3jbA==}
engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
peerDependencies:
eslint: ^6.0.0 || ^7.0.0 || >=8.0.0
@@ -888,8 +756,8 @@ packages:
eslint-visitor-keys: 3.4.3
dev: true
- /@eslint-community/regexpp@4.11.0:
- resolution: {integrity: sha512-G/M/tIiMrTAxEWRfLfQJMmGNX28IxBg4PBz8XqQhqUHLFI6TL2htpIB1iQCj144V5ee/JaKyT9/WZ0MGZWfA7A==}
+ /@eslint-community/regexpp@4.11.2:
+ resolution: {integrity: sha512-2WwyTYNVaMNUWPZTOJdkax9iqTdirrApgTbk+Qoq5EPX6myqZvG8QGFRgdKmkjKVG6/G/a565vpPauHk0+hpBA==}
engines: {node: ^12.0.0 || ^14.0.0 || >=16.0.0}
dev: true
@@ -898,7 +766,7 @@ packages:
engines: {node: ^10.12.0 || >=12.0.0}
dependencies:
ajv: 6.12.6
- debug: 4.3.5
+ debug: 4.3.7
espree: 7.3.1
globals: 13.24.0
ignore: 4.0.6
@@ -920,7 +788,7 @@ packages:
deprecated: Use @eslint/config-array instead
dependencies:
'@humanwhocodes/object-schema': 1.2.1
- debug: 4.3.5
+ debug: 4.3.7
minimatch: 3.1.2
transitivePeerDependencies:
- supports-color
@@ -1003,142 +871,128 @@ packages:
dev: true
optional: true
- /@rollup/pluginutils@5.1.0:
- resolution: {integrity: sha512-XTIWOPPcpvyKI6L1NHo0lFlCyznUEyPmPY1mc3KpPVDYulHSTvyeLNVW00QTLIAFNhR3kYnJTQHeGqU4M3n09g==}
- engines: {node: '>=14.0.0'}
- peerDependencies:
- rollup: ^1.20.0||^2.0.0||^3.0.0||^4.0.0
- peerDependenciesMeta:
- rollup:
- optional: true
- dependencies:
- '@types/estree': 1.0.5
- estree-walker: 2.0.2
- picomatch: 2.3.1
- dev: true
-
- /@rollup/rollup-android-arm-eabi@4.18.1:
- resolution: {integrity: sha512-lncuC4aHicncmbORnx+dUaAgzee9cm/PbIqgWz1PpXuwc+sa1Ct83tnqUDy/GFKleLiN7ZIeytM6KJ4cAn1SxA==}
+ /@rollup/rollup-android-arm-eabi@4.24.0:
+ resolution: {integrity: sha512-Q6HJd7Y6xdB48x8ZNVDOqsbh2uByBhgK8PiQgPhwkIw/HC/YX5Ghq2mQY5sRMZWHb3VsFkWooUVOZHKr7DmDIA==}
cpu: [arm]
os: [android]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-android-arm64@4.18.1:
- resolution: {integrity: sha512-F/tkdw0WSs4ojqz5Ovrw5r9odqzFjb5LIgHdHZG65dFI1lWTWRVy32KDJLKRISHgJvqUeUhdIvy43fX41znyDg==}
+ /@rollup/rollup-android-arm64@4.24.0:
+ resolution: {integrity: sha512-ijLnS1qFId8xhKjT81uBHuuJp2lU4x2yxa4ctFPtG+MqEE6+C5f/+X/bStmxapgmwLwiL3ih122xv8kVARNAZA==}
cpu: [arm64]
os: [android]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-darwin-arm64@4.18.1:
- resolution: {integrity: sha512-vk+ma8iC1ebje/ahpxpnrfVQJibTMyHdWpOGZ3JpQ7Mgn/3QNHmPq7YwjZbIE7km73dH5M1e6MRRsnEBW7v5CQ==}
+ /@rollup/rollup-darwin-arm64@4.24.0:
+ resolution: {integrity: sha512-bIv+X9xeSs1XCk6DVvkO+S/z8/2AMt/2lMqdQbMrmVpgFvXlmde9mLcbQpztXm1tajC3raFDqegsH18HQPMYtA==}
cpu: [arm64]
os: [darwin]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-darwin-x64@4.18.1:
- resolution: {integrity: sha512-IgpzXKauRe1Tafcej9STjSSuG0Ghu/xGYH+qG6JwsAUxXrnkvNHcq/NL6nz1+jzvWAnQkuAJ4uIwGB48K9OCGA==}
+ /@rollup/rollup-darwin-x64@4.24.0:
+ resolution: {integrity: sha512-X6/nOwoFN7RT2svEQWUsW/5C/fYMBe4fnLK9DQk4SX4mgVBiTA9h64kjUYPvGQ0F/9xwJ5U5UfTbl6BEjaQdBQ==}
cpu: [x64]
os: [darwin]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-arm-gnueabihf@4.18.1:
- resolution: {integrity: sha512-P9bSiAUnSSM7EmyRK+e5wgpqai86QOSv8BwvkGjLwYuOpaeomiZWifEos517CwbG+aZl1T4clSE1YqqH2JRs+g==}
+ /@rollup/rollup-linux-arm-gnueabihf@4.24.0:
+ resolution: {integrity: sha512-0KXvIJQMOImLCVCz9uvvdPgfyWo93aHHp8ui3FrtOP57svqrF/roSSR5pjqL2hcMp0ljeGlU4q9o/rQaAQ3AYA==}
cpu: [arm]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-arm-musleabihf@4.18.1:
- resolution: {integrity: sha512-5RnjpACoxtS+aWOI1dURKno11d7krfpGDEn19jI8BuWmSBbUC4ytIADfROM1FZrFhQPSoP+KEa3NlEScznBTyQ==}
+ /@rollup/rollup-linux-arm-musleabihf@4.24.0:
+ resolution: {integrity: sha512-it2BW6kKFVh8xk/BnHfakEeoLPv8STIISekpoF+nBgWM4d55CZKc7T4Dx1pEbTnYm/xEKMgy1MNtYuoA8RFIWw==}
cpu: [arm]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-arm64-gnu@4.18.1:
- resolution: {integrity: sha512-8mwmGD668m8WaGbthrEYZ9CBmPug2QPGWxhJxh/vCgBjro5o96gL04WLlg5BA233OCWLqERy4YUzX3bJGXaJgQ==}
+ /@rollup/rollup-linux-arm64-gnu@4.24.0:
+ resolution: {integrity: sha512-i0xTLXjqap2eRfulFVlSnM5dEbTVque/3Pi4g2y7cxrs7+a9De42z4XxKLYJ7+OhE3IgxvfQM7vQc43bwTgPwA==}
cpu: [arm64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-arm64-musl@4.18.1:
- resolution: {integrity: sha512-dJX9u4r4bqInMGOAQoGYdwDP8lQiisWb9et+T84l2WXk41yEej8v2iGKodmdKimT8cTAYt0jFb+UEBxnPkbXEQ==}
+ /@rollup/rollup-linux-arm64-musl@4.24.0:
+ resolution: {integrity: sha512-9E6MKUJhDuDh604Qco5yP/3qn3y7SLXYuiC0Rpr89aMScS2UAmK1wHP2b7KAa1nSjWJc/f/Lc0Wl1L47qjiyQw==}
cpu: [arm64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-powerpc64le-gnu@4.18.1:
- resolution: {integrity: sha512-V72cXdTl4EI0x6FNmho4D502sy7ed+LuVW6Ym8aI6DRQ9hQZdp5sj0a2usYOlqvFBNKQnLQGwmYnujo2HvjCxQ==}
+ /@rollup/rollup-linux-powerpc64le-gnu@4.24.0:
+ resolution: {integrity: sha512-2XFFPJ2XMEiF5Zi2EBf4h73oR1V/lycirxZxHZNc93SqDN/IWhYYSYj8I9381ikUFXZrz2v7r2tOVk2NBwxrWw==}
cpu: [ppc64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-riscv64-gnu@4.18.1:
- resolution: {integrity: sha512-f+pJih7sxoKmbjghrM2RkWo2WHUW8UbfxIQiWo5yeCaCM0TveMEuAzKJte4QskBp1TIinpnRcxkquY+4WuY/tg==}
+ /@rollup/rollup-linux-riscv64-gnu@4.24.0:
+ resolution: {integrity: sha512-M3Dg4hlwuntUCdzU7KjYqbbd+BLq3JMAOhCKdBE3TcMGMZbKkDdJ5ivNdehOssMCIokNHFOsv7DO4rlEOfyKpg==}
cpu: [riscv64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-s390x-gnu@4.18.1:
- resolution: {integrity: sha512-qb1hMMT3Fr/Qz1OKovCuUM11MUNLUuHeBC2DPPAWUYYUAOFWaxInaTwTQmc7Fl5La7DShTEpmYwgdt2hG+4TEg==}
+ /@rollup/rollup-linux-s390x-gnu@4.24.0:
+ resolution: {integrity: sha512-mjBaoo4ocxJppTorZVKWFpy1bfFj9FeCMJqzlMQGjpNPY9JwQi7OuS1axzNIk0nMX6jSgy6ZURDZ2w0QW6D56g==}
cpu: [s390x]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-x64-gnu@4.18.1:
- resolution: {integrity: sha512-7O5u/p6oKUFYjRbZkL2FLbwsyoJAjyeXHCU3O4ndvzg2OFO2GinFPSJFGbiwFDaCFc+k7gs9CF243PwdPQFh5g==}
+ /@rollup/rollup-linux-x64-gnu@4.24.0:
+ resolution: {integrity: sha512-ZXFk7M72R0YYFN5q13niV0B7G8/5dcQ9JDp8keJSfr3GoZeXEoMHP/HlvqROA3OMbMdfr19IjCeNAnPUG93b6A==}
cpu: [x64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-linux-x64-musl@4.18.1:
- resolution: {integrity: sha512-pDLkYITdYrH/9Cv/Vlj8HppDuLMDUBmgsM0+N+xLtFd18aXgM9Nyqupb/Uw+HeidhfYg2lD6CXvz6CjoVOaKjQ==}
+ /@rollup/rollup-linux-x64-musl@4.24.0:
+ resolution: {integrity: sha512-w1i+L7kAXZNdYl+vFvzSZy8Y1arS7vMgIy8wusXJzRrPyof5LAb02KGr1PD2EkRcl73kHulIID0M501lN+vobQ==}
cpu: [x64]
os: [linux]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-win32-arm64-msvc@4.18.1:
- resolution: {integrity: sha512-W2ZNI323O/8pJdBGil1oCauuCzmVd9lDmWBBqxYZcOqWD6aWqJtVBQ1dFrF4dYpZPks6F+xCZHfzG5hYlSHZ6g==}
+ /@rollup/rollup-win32-arm64-msvc@4.24.0:
+ resolution: {integrity: sha512-VXBrnPWgBpVDCVY6XF3LEW0pOU51KbaHhccHw6AS6vBWIC60eqsH19DAeeObl+g8nKAz04QFdl/Cefta0xQtUQ==}
cpu: [arm64]
os: [win32]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-win32-ia32-msvc@4.18.1:
- resolution: {integrity: sha512-ELfEX1/+eGZYMaCIbK4jqLxO1gyTSOIlZr6pbC4SRYFaSIDVKOnZNMdoZ+ON0mrFDp4+H5MhwNC1H/AhE3zQLg==}
+ /@rollup/rollup-win32-ia32-msvc@4.24.0:
+ resolution: {integrity: sha512-xrNcGDU0OxVcPTH/8n/ShH4UevZxKIO6HJFK0e15XItZP2UcaiLFd5kiX7hJnqCbSztUF8Qot+JWBC/QXRPYWQ==}
cpu: [ia32]
os: [win32]
requiresBuild: true
dev: true
optional: true
- /@rollup/rollup-win32-x64-msvc@4.18.1:
- resolution: {integrity: sha512-yjk2MAkQmoaPYCSu35RLJ62+dz358nE83VfTePJRp8CG7aMg25mEJYpXFiD+NcevhX8LxD5OP5tktPXnXN7GDw==}
+ /@rollup/rollup-win32-x64-msvc@4.24.0:
+ resolution: {integrity: sha512-fbMkAF7fufku0N2dE5TBXcNlg0pt0cJue4xBRE2Qc5Vqikxr4VCgKj/ht6SMdFcOacVA9rqF70APJ8RN/4vMJw==}
cpu: [x64]
os: [win32]
requiresBuild: true
@@ -1154,130 +1008,129 @@ packages:
engines: {node: '>=6'}
dev: true
- /@svgr/babel-plugin-add-jsx-attribute@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-b9MIk7yhdS1pMCZM8VeNfUlSKVRhsHZNMl5O9SfaX0l0t5wjdgu4IDzGB8bpnGBBOjGST3rRFVsaaEtI4W6f7g==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-darwin-arm64@1.7.40:
+ resolution: {integrity: sha512-LRRrCiRJLb1kpQtxMNNsr5W82Inr0dy5Imho+4HQzVx/Ismi0qX4hQBgzJAnyOBNLK1+OBVb/912UVhKXppdfQ==}
+ engines: {node: '>=10'}
+ cpu: [arm64]
+ os: [darwin]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-remove-jsx-attribute@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-BcCkm/STipKvbCl6b7QFrMh/vx00vIP63k2eM66MfHJzPr6O2U0jYEViXkHJWqXqQYjdeA9cuCl5KWmlwjDvbA==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-darwin-x64@1.7.40:
+ resolution: {integrity: sha512-Lpl0XK/4fLzS5jsK48opUuGXrqJXwqJckYYPwyGbCfCXm4MsBe+7dX2hq/Kc4YMY25+NeTmzAXhla8TT4WYD/g==}
+ engines: {node: '>=10'}
+ cpu: [x64]
+ os: [darwin]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-remove-jsx-empty-expression@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-5BcGCBfBxB5+XSDSWnhTThfI9jcO5f0Ai2V24gZpG+wXF14BzwxxdDb4g6trdOux0rhibGs385BeFMSmxtS3uA==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-linux-arm-gnueabihf@1.7.40:
+ resolution: {integrity: sha512-4bEvvjptpoc5BRPr/R419h6fXTEuub+frpxxlxBOEKxgXjAF/S3xdxyPijUAakmW/xXBF0u7OC4KYI+38yQp6g==}
+ engines: {node: '>=10'}
+ cpu: [arm]
+ os: [linux]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-replace-jsx-attribute-value@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-KVQ+PtIjb1BuYT3ht8M5KbzWBhdAjjUPdlMtpuw/VjT8coTrItWX6Qafl9+ji831JaJcu6PJNKCV0bp01lBNzQ==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-linux-arm64-gnu@1.7.40:
+ resolution: {integrity: sha512-v2fBlHJ/6Ovz0L2xFAI9TRiKyl9DTdx139PuAHD9gyzp16Utl/W0MPd4t2cYdkI6hPXE9PsJCSzMOrduh+YoDg==}
+ engines: {node: '>=10'}
+ cpu: [arm64]
+ os: [linux]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-svg-dynamic-title@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-omNiKqwjNmOQJ2v6ge4SErBbkooV2aAWwaPFs2vUY7p7GhVkzRkJ00kILXQvRhA6miHnNpXv7MRnnSjdRjK8og==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-linux-arm64-musl@1.7.40:
+ resolution: {integrity: sha512-uMkduQuU4LFVkW6txv8AVArT8GjJVJ5IHoWloXaUBMT447iE8NALmpePdZWhMyj6KV7j0y23CM5rzV/I2eNGLg==}
+ engines: {node: '>=10'}
+ cpu: [arm64]
+ os: [linux]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-svg-em-dimensions@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-mURHYnu6Iw3UBTbhGwE/vsngtCIbHE43xCRK7kCw4t01xyGqb2Pd+WXekRRoFOBIY29ZoOhUCTEweDMdrjfi9g==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-linux-x64-gnu@1.7.40:
+ resolution: {integrity: sha512-4LZdY1MBSnXyTpW5fpBU/+JGAhkuHT+VnFTDNegRboN5nSPh7y0Yvn4LmIioESV+sWzjKkEXujJPGjrp+oSp5w==}
+ engines: {node: '>=10'}
+ cpu: [x64]
+ os: [linux]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-transform-react-native-svg@8.1.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-Tx8T58CHo+7nwJ+EhUwx3LfdNSG9R2OKfaIXXs5soiy5HtgoAEkDay9LIimLOcG8dJQH1wPZp/cnAv6S9CrR1Q==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-linux-x64-musl@1.7.40:
+ resolution: {integrity: sha512-FPjOwT3SgI6PAwH1O8bhOGBPzuvzOlzKeCtxLaCjruHJu9V8KKBrMTWOZT/FJyYC9mX5Ip1+l9j30UqUZdQxtA==}
+ engines: {node: '>=10'}
+ cpu: [x64]
+ os: [linux]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-plugin-transform-svg-component@8.0.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-DFx8xa3cZXTdb/k3kfPeaixecQLgKh5NVBMwD0AQxOzcZawK4oo1Jh9LbrcACUivsCA7TLG8eeWgrDXjTMhRmw==}
- engines: {node: '>=12'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
+ /@swc/core-win32-arm64-msvc@1.7.40:
+ resolution: {integrity: sha512-//ovXdD9GsTmhPmXJlXnIbRQkeuL6PSrYSr7uCMNcclrUdJG0YkO0GMM2afUKYbdJcunylDDWsSS8PFWn0QxmA==}
+ engines: {node: '>=10'}
+ cpu: [arm64]
+ os: [win32]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/babel-preset@8.1.0(@babel/core@7.24.8):
- resolution: {integrity: sha512-7EYDbHE7MxHpv4sxvnVPngw5fuR6pw79SkcrILHJ/iMpuKySNCl5W1qcwPEpU+LgyRXOaAFgH0KhwD18wwg6ug==}
- engines: {node: '>=14'}
- peerDependencies:
- '@babel/core': ^7.0.0-0
- dependencies:
- '@babel/core': 7.24.8
- '@svgr/babel-plugin-add-jsx-attribute': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-remove-jsx-attribute': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-remove-jsx-empty-expression': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-replace-jsx-attribute-value': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-svg-dynamic-title': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-svg-em-dimensions': 8.0.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-transform-react-native-svg': 8.1.0(@babel/core@7.24.8)
- '@svgr/babel-plugin-transform-svg-component': 8.0.0(@babel/core@7.24.8)
+ /@swc/core-win32-ia32-msvc@1.7.40:
+ resolution: {integrity: sha512-iD/1auVhHGlhWAPrWmfRWL3w4AvXIWGVXZiSA109/xnRIPiHKb/HqqTp/qB94E/ZHMPRgLKkLTNwamlkueUs8g==}
+ engines: {node: '>=10'}
+ cpu: [ia32]
+ os: [win32]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/core@8.1.0(typescript@4.9.5):
- resolution: {integrity: sha512-8QqtOQT5ACVlmsvKOJNEaWmRPmcojMOzCz4Hs2BGG/toAp/K38LcsMRyLp349glq5AzJbCEeimEoxaX6v/fLrA==}
- engines: {node: '>=14'}
- dependencies:
- '@babel/core': 7.24.8
- '@svgr/babel-preset': 8.1.0(@babel/core@7.24.8)
- camelcase: 6.3.0
- cosmiconfig: 8.3.6(typescript@4.9.5)
- snake-case: 3.0.4
- transitivePeerDependencies:
- - supports-color
- - typescript
+ /@swc/core-win32-x64-msvc@1.7.40:
+ resolution: {integrity: sha512-ZlFAV1WFPhhWQ/8esiygmetkb905XIcMMtHRRG0FBGCllO+HVL5nikUaLDgTClz1onmEY9sMXUFQeoPtvliV+w==}
+ engines: {node: '>=10'}
+ cpu: [x64]
+ os: [win32]
+ requiresBuild: true
dev: true
+ optional: true
- /@svgr/hast-util-to-babel-ast@8.0.0:
- resolution: {integrity: sha512-EbDKwO9GpfWP4jN9sGdYwPBU0kdomaPIL2Eu4YwmgP+sJeXT+L7bMwJUBnhzfH8Q2qMBqZ4fJwpCyYsAN3mt2Q==}
- engines: {node: '>=14'}
+ /@swc/core@1.7.40:
+ resolution: {integrity: sha512-0HIzM5vigVT5IvNum+pPuST9p8xFhN6mhdIKju7qYYeNuZG78lwms/2d8WgjTJJlzp6JlPguXGrMMNzjQw0qNg==}
+ engines: {node: '>=10'}
+ requiresBuild: true
+ peerDependencies:
+ '@swc/helpers': '*'
+ peerDependenciesMeta:
+ '@swc/helpers':
+ optional: true
dependencies:
- '@babel/types': 7.24.8
- entities: 4.5.0
+ '@swc/counter': 0.1.3
+ '@swc/types': 0.1.13
+ optionalDependencies:
+ '@swc/core-darwin-arm64': 1.7.40
+ '@swc/core-darwin-x64': 1.7.40
+ '@swc/core-linux-arm-gnueabihf': 1.7.40
+ '@swc/core-linux-arm64-gnu': 1.7.40
+ '@swc/core-linux-arm64-musl': 1.7.40
+ '@swc/core-linux-x64-gnu': 1.7.40
+ '@swc/core-linux-x64-musl': 1.7.40
+ '@swc/core-win32-arm64-msvc': 1.7.40
+ '@swc/core-win32-ia32-msvc': 1.7.40
+ '@swc/core-win32-x64-msvc': 1.7.40
dev: true
- /@svgr/plugin-jsx@8.1.0(@svgr/core@8.1.0):
- resolution: {integrity: sha512-0xiIyBsLlr8quN+WyuxooNW9RJ0Dpr8uOnH/xrCVO8GLUcwHISwj1AG0k+LFzteTkAA0GbX0kj9q6Dk70PTiPA==}
- engines: {node: '>=14'}
- peerDependencies:
- '@svgr/core': '*'
+ /@swc/counter@0.1.3:
+ resolution: {integrity: sha512-e2BR4lsJkkRlKZ/qCHPw9ZaSxc0MVUd7gtbtaB7aMvHeJVYe8sOB8DBZkP2DtISHGSku9sCK6T6cnY0CtXrOCQ==}
+ dev: true
+
+ /@swc/types@0.1.13:
+ resolution: {integrity: sha512-JL7eeCk6zWCbiYQg2xQSdLXQJl8Qoc9rXmG2cEKvHe3CKwMHwHGpfOb8frzNLmbycOo6I51qxnLnn9ESf4I20Q==}
dependencies:
- '@babel/core': 7.24.8
- '@svgr/babel-preset': 8.1.0(@babel/core@7.24.8)
- '@svgr/core': 8.1.0(typescript@4.9.5)
- '@svgr/hast-util-to-babel-ast': 8.0.0
- svg-parser: 2.0.4
- transitivePeerDependencies:
- - supports-color
+ '@swc/counter': 0.1.3
dev: true
/@szmarczak/http-timer@1.1.2:
@@ -1287,37 +1140,8 @@ packages:
defer-to-connect: 1.1.3
dev: true
- /@types/babel__core@7.20.5:
- resolution: {integrity: sha512-qoQprZvz5wQFJwMDqeseRXWv3rqMvhgpbXFfVyWhbx9X47POIA6i/+dXefEmZKoAgOaTdaIgNSMqMIU61yRyzA==}
- dependencies:
- '@babel/parser': 7.24.8
- '@babel/types': 7.24.8
- '@types/babel__generator': 7.6.8
- '@types/babel__template': 7.4.4
- '@types/babel__traverse': 7.20.6
- dev: true
-
- /@types/babel__generator@7.6.8:
- resolution: {integrity: sha512-ASsj+tpEDsEiFr1arWrlN6V3mdfjRMZt6LtK/Vp/kreFLnr5QH5+DhvD5nINYZXzwJvXeGq+05iUXcAzVrqWtw==}
- dependencies:
- '@babel/types': 7.24.8
- dev: true
-
- /@types/babel__template@7.4.4:
- resolution: {integrity: sha512-h/NUaSyG5EyxBIp8YRxo4RMe2/qQgvyowRwVMzhYhBCONbW8PUsg4lkFMrhgZhUe5z3L3MiLDuvyJ/CaPa2A8A==}
- dependencies:
- '@babel/parser': 7.24.8
- '@babel/types': 7.24.8
- dev: true
-
- /@types/babel__traverse@7.20.6:
- resolution: {integrity: sha512-r1bzfrm0tomOI8g1SzvCaQHo6Lcv6zu0EA+W2kHrt8dyrHQxGzBBL4kdkzIS+jBMV+EYcMAEAqXqYaLJq5rOZg==}
- dependencies:
- '@babel/types': 7.24.8
- dev: true
-
- /@types/estree@1.0.5:
- resolution: {integrity: sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==}
+ /@types/estree@1.0.6:
+ resolution: {integrity: sha512-AYnb1nQyY49te+VRAVgmzfcgjYS91mY5P0TKUDCLEM+gNnA+3T6rWITXRLYCpahpqSQbN5cE+gHpnPyXjHWxcw==}
dev: true
/@types/history@4.7.11:
@@ -1338,21 +1162,21 @@ packages:
/@types/keyv@3.1.4:
resolution: {integrity: sha512-BQ5aZNSCpj7D6K2ksrRCTmKRLEpnPvWDiLPfoGyhZ++8YtiK9d/3DBKPJgry359X/P1PfruyYwvnvwFjuEiEIg==}
dependencies:
- '@types/node': 20.14.10
+ '@types/node': 22.8.1
dev: true
- /@types/node@20.14.10:
- resolution: {integrity: sha512-MdiXf+nDuMvY0gJKxyfZ7/6UFsETO7mGKF54MVD/ekJS6HdFtpZFBgrh6Pseu64XTb2MLyFPlbW6hj8HYRQNOQ==}
+ /@types/node@22.8.1:
+ resolution: {integrity: sha512-k6Gi8Yyo8EtrNtkHXutUu2corfDf9su95VYVP10aGYMMROM6SAItZi0w1XszA6RtWTHSVp5OeFof37w0IEqCQg==}
dependencies:
- undici-types: 5.26.5
+ undici-types: 6.19.8
dev: true
/@types/parse-json@4.0.2:
resolution: {integrity: sha512-dISoDXWWQwUquiKsyZ4Ng+HX2KsPL7LyHKHQwgGFEA3IaKac4Obd+h2a/a6waisAoepJlBcx9paWqjA8/HVjCw==}
dev: false
- /@types/prop-types@15.7.12:
- resolution: {integrity: sha512-5zvhXYtRNRluoE/jAp4GVsSduVUzNWKkOZrCDBWYtE7biZywwdC2AcEzg+cSMLFRfVgeAFqpfNabiPjxFddV1Q==}
+ /@types/prop-types@15.7.13:
+ resolution: {integrity: sha512-hCZTSvwbzWGvhqxp/RqVqwU999pBf2vp7hzIjiYOsl8wqOmUxkQ6ddw1cV3l8811+kdUFus/q4d1Y3E3SyEifA==}
dev: true
/@types/react-dom@16.8.4:
@@ -1381,11 +1205,11 @@ packages:
dependencies:
'@types/react': 16.8.15
'@types/react-dom': 16.8.4
- '@types/react-transition-group': 4.4.10
+ '@types/react-transition-group': 4.4.11
dev: true
- /@types/react-transition-group@4.4.10:
- resolution: {integrity: sha512-hT/+s0VQs2ojCX823m60m5f0sL5idt9SO6Tj6Dg+rdphGPIeJbJ6CxvBYkgkGKrYeDjvIpKTR38UzmtHJOGW3Q==}
+ /@types/react-transition-group@4.4.11:
+ resolution: {integrity: sha512-RM05tAniPZ5DZPzzNFP+DmrcOdD0efDUxMy3145oljWSl3x9ZV5vhme98gTxFrj2lhXvmGNnUiuDyJgY9IKkNA==}
dependencies:
'@types/react': 16.8.15
dev: true
@@ -1393,14 +1217,14 @@ packages:
/@types/react@16.8.15:
resolution: {integrity: sha512-dMhzw1rWK+wwJWvPp5Pk12ksSrm/z/C/+lOQbMZ7YfDQYnJ02bc0wtg4EJD9qrFhuxFrf/ywNgwTboucobJqQg==}
dependencies:
- '@types/prop-types': 15.7.12
+ '@types/prop-types': 15.7.13
csstype: 2.6.21
dev: true
/@types/responselike@1.0.3:
resolution: {integrity: sha512-H/+L+UkTV33uf49PH5pCAUBVPNj2nDBXTN+qS1dOwyyg24l3CcicicCA7ca+HMvJBZcFgl5r8e+RR6elsb4Lyw==}
dependencies:
- '@types/node': 20.14.10
+ '@types/node': 22.8.1
dev: true
/@types/semver@7.5.8:
@@ -1418,17 +1242,17 @@ packages:
typescript:
optional: true
dependencies:
- '@eslint-community/regexpp': 4.11.0
+ '@eslint-community/regexpp': 4.11.2
'@typescript-eslint/parser': 5.62.0(eslint@7.32.0)(typescript@4.9.5)
'@typescript-eslint/scope-manager': 5.62.0
'@typescript-eslint/type-utils': 5.62.0(eslint@7.32.0)(typescript@4.9.5)
'@typescript-eslint/utils': 5.62.0(eslint@7.32.0)(typescript@4.9.5)
- debug: 4.3.5
+ debug: 4.3.7
eslint: 7.32.0
graphemer: 1.4.0
- ignore: 5.3.1
+ ignore: 5.3.2
natural-compare-lite: 1.4.0
- semver: 7.6.2
+ semver: 7.6.3
tsutils: 3.21.0(typescript@4.9.5)
typescript: 4.9.5
transitivePeerDependencies:
@@ -1448,7 +1272,7 @@ packages:
'@typescript-eslint/scope-manager': 5.62.0
'@typescript-eslint/types': 5.62.0
'@typescript-eslint/typescript-estree': 5.62.0(typescript@4.9.5)
- debug: 4.3.5
+ debug: 4.3.7
eslint: 7.32.0
typescript: 4.9.5
transitivePeerDependencies:
@@ -1475,7 +1299,7 @@ packages:
dependencies:
'@typescript-eslint/typescript-estree': 5.62.0(typescript@4.9.5)
'@typescript-eslint/utils': 5.62.0(eslint@7.32.0)(typescript@4.9.5)
- debug: 4.3.5
+ debug: 4.3.7
eslint: 7.32.0
tsutils: 3.21.0(typescript@4.9.5)
typescript: 4.9.5
@@ -1499,10 +1323,10 @@ packages:
dependencies:
'@typescript-eslint/types': 5.62.0
'@typescript-eslint/visitor-keys': 5.62.0
- debug: 4.3.5
+ debug: 4.3.7
globby: 11.1.0
is-glob: 4.0.3
- semver: 7.6.2
+ semver: 7.6.3
tsutils: 3.21.0(typescript@4.9.5)
typescript: 4.9.5
transitivePeerDependencies:
@@ -1515,7 +1339,7 @@ packages:
peerDependencies:
eslint: ^6.0.0 || ^7.0.0 || ^8.0.0
dependencies:
- '@eslint-community/eslint-utils': 4.4.0(eslint@7.32.0)
+ '@eslint-community/eslint-utils': 4.4.1(eslint@7.32.0)
'@types/json-schema': 7.0.15
'@types/semver': 7.5.8
'@typescript-eslint/scope-manager': 5.62.0
@@ -1523,7 +1347,7 @@ packages:
'@typescript-eslint/typescript-estree': 5.62.0(typescript@4.9.5)
eslint: 7.32.0
eslint-scope: 5.1.1
- semver: 7.6.2
+ semver: 7.6.3
transitivePeerDependencies:
- supports-color
- typescript
@@ -1537,20 +1361,15 @@ packages:
eslint-visitor-keys: 3.4.3
dev: true
- /@vitejs/plugin-react@4.3.1(vite@4.5.3):
- resolution: {integrity: sha512-m/V2syj5CuVnaxcUJOQRel/Wr31FFXRFlnOoq1TVtkCxsY5veGMTEmpWHndrhB2U8ScHtCQB1e+4hWYExQc6Lg==}
- engines: {node: ^14.18.0 || >=16.0.0}
+ /@vitejs/plugin-react-swc@3.7.1(vite@4.5.3):
+ resolution: {integrity: sha512-vgWOY0i1EROUK0Ctg1hwhtC3SdcDjZcdit4Ups4aPkDcB1jYhmo+RMYWY87cmXMhvtD5uf8lV89j2w16vkdSVg==}
peerDependencies:
- vite: ^4.2.0 || ^5.0.0
+ vite: ^4 || ^5
dependencies:
- '@babel/core': 7.24.8
- '@babel/plugin-transform-react-jsx-self': 7.24.7(@babel/core@7.24.8)
- '@babel/plugin-transform-react-jsx-source': 7.24.7(@babel/core@7.24.8)
- '@types/babel__core': 7.20.5
- react-refresh: 0.14.2
+ '@swc/core': 1.7.40
vite: 4.5.3(less@3.13.1)
transitivePeerDependencies:
- - supports-color
+ - '@swc/helpers'
dev: true
/@vitest/expect@1.6.0:
@@ -1558,7 +1377,7 @@ packages:
dependencies:
'@vitest/spy': 1.6.0
'@vitest/utils': 1.6.0
- chai: 4.4.1
+ chai: 4.5.0
dev: true
/@vitest/runner@1.6.0:
@@ -1572,7 +1391,7 @@ packages:
/@vitest/snapshot@1.6.0:
resolution: {integrity: sha512-+Hx43f8Chus+DCmygqqfetcAZrDJwvTj0ymqjQq4CvmpKFSTVteEOBzCusu1x2tt4OJcvBflyHUE0DZSLgEMtQ==}
dependencies:
- magic-string: 0.30.10
+ magic-string: 0.30.12
pathe: 1.1.2
pretty-format: 29.7.0
dev: true
@@ -1608,11 +1427,11 @@ packages:
acorn: 7.4.1
dev: true
- /acorn-walk@8.3.3:
- resolution: {integrity: sha512-MxXdReSRhGO7VlFe1bRG/oI7/mdLV9B9JJT0N8vZOhF7gFRR5l3M8W9G8JxmKV+JC5mGqJ0QvqfSOLsCPa4nUw==}
+ /acorn-walk@8.3.4:
+ resolution: {integrity: sha512-ueEepnujpqee2o5aIYnvHU6C0A42MNdsIDeqy5BydrkuC5R1ZuUFnm27EeFJGoEHJQgn3uleRvmTXaJgfXbt4g==}
engines: {node: '>=0.4.0'}
dependencies:
- acorn: 8.12.1
+ acorn: 8.13.0
dev: true
/acorn@7.4.1:
@@ -1621,8 +1440,8 @@ packages:
hasBin: true
dev: true
- /acorn@8.12.1:
- resolution: {integrity: sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==}
+ /acorn@8.13.0:
+ resolution: {integrity: sha512-8zSiw54Oxrdym50NlZ9sUusyO1Z1ZchgRLWRaK6c86XJFClyCgFKetdowBg5bKxyp/u+CDBJG4Mpp0m3HLZl9w==}
engines: {node: '>=0.4.0'}
hasBin: true
dev: true
@@ -1653,13 +1472,13 @@ packages:
uri-js: 4.4.1
dev: true
- /ajv@8.16.0:
- resolution: {integrity: sha512-F0twR8U1ZU67JIEtekUcLkXkoO5mMMmgGD8sK/xUFzJ805jxHQl92hImFAqqXMyMYjSPOyUPAwHYhB72g5sTXw==}
+ /ajv@8.17.1:
+ resolution: {integrity: sha512-B/gBuNg5SiMTrPkC+A2+cW0RszwxYmn6VYxB/inlBStS5nx6xHIt/ehKRhIMhqusl7a8LjQoZnjCs5vhwxOQ1g==}
dependencies:
fast-deep-equal: 3.1.3
+ fast-uri: 3.0.3
json-schema-traverse: 1.0.0
require-from-string: 2.0.2
- uri-js: 4.4.1
dev: true
/ansi-align@3.0.1:
@@ -1688,8 +1507,8 @@ packages:
engines: {node: '>=8'}
dev: true
- /ansi-regex@6.0.1:
- resolution: {integrity: sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA==}
+ /ansi-regex@6.1.0:
+ resolution: {integrity: sha512-7HSX4QQb4CspciLpVFwyRe79O3xsIZDDLER21kERQ71oaPodF8jL725AgJMFAYbooIqolJoRLuM81SpeUkpkvA==}
engines: {node: '>=12'}
dev: true
@@ -1698,6 +1517,7 @@ packages:
engines: {node: '>=4'}
dependencies:
color-convert: 1.9.3
+ dev: true
/ansi-styles@4.3.0:
resolution: {integrity: sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==}
@@ -1725,7 +1545,7 @@ packages:
'@ant-design/colors': 5.1.1
'@ant-design/icons': 4.8.3(react-dom@16.14.0)(react@16.14.0)
'@ant-design/react-slick': 0.28.4(react@16.14.0)
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
array-tree-filter: 2.1.0
classnames: 2.5.1
copy-to-clipboard: 3.3.3
@@ -1742,7 +1562,7 @@ packages:
rc-input-number: 6.1.3(react-dom@16.14.0)(react@16.14.0)
rc-mentions: 1.5.3(react-dom@16.14.0)(react@16.14.0)
rc-menu: 8.10.8(react-dom@16.14.0)(react@16.14.0)
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-notification: 4.5.7(react-dom@16.14.0)(react@16.14.0)
rc-pagination: 3.1.17(react-dom@16.14.0)(react@16.14.0)
rc-picker: 2.5.19(react-dom@16.14.0)(react@16.14.0)
@@ -1777,10 +1597,6 @@ packages:
sprintf-js: 1.0.3
dev: true
- /argparse@2.0.1:
- resolution: {integrity: sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==}
- dev: true
-
/array-buffer-byte-length@1.0.1:
resolution: {integrity: sha512-ahC5W1xgou+KTXix4sAO8Ki12Q+jf4i0+tmk3sC+zgcynshkHxzpXdImBehiUYKKKDwvfFiJl1tZt6ewscS1Mg==}
engines: {node: '>= 0.4'}
@@ -1854,15 +1670,15 @@ packages:
resolution: {integrity: sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==}
dev: true
- /aws4@1.13.0:
- resolution: {integrity: sha512-3AungXC4I8kKsS9PuS4JH2nc+0bVY/mjgrephHTIi8fpEeGsTHBUJeosp0Wc1myYMElmD0B3Oc4XL/HVJ4PV2g==}
+ /aws4@1.13.2:
+ resolution: {integrity: sha512-lHe62zvbTB5eEABUVi/AwVh0ZKY9rMMDhmm+eeyuuUQbQ3+J+fONVQOZyj+DdrvD4BY33uYniyRJ4UJIaSKAfw==}
dev: true
/axios@0.28.1:
resolution: {integrity: sha512-iUcGA5a7p0mVb4Gm/sy+FSECNkPFT4y7wt6OM/CDpO/OnNCvSs3PoMG8ibrC9jRoGYU0gUK5pXVC4NPXq6lHRQ==}
dependencies:
- follow-redirects: 1.15.6
- form-data: 4.0.0
+ follow-redirects: 1.15.9
+ form-data: 4.0.1
proxy-from-env: 1.1.0
transitivePeerDependencies:
- debug
@@ -1871,7 +1687,7 @@ packages:
/babel-plugin-emotion@10.2.2:
resolution: {integrity: sha512-SMSkGoqTbTyUTDeuVuPIWifPdUGkTk1Kf9BWRiXIOIcuyMfsdp2EjeiiFvOzX8NOBvEh/ypKYvUh2rkgAJMCLA==}
dependencies:
- '@babel/helper-module-imports': 7.24.7
+ '@babel/helper-module-imports': 7.25.9
'@emotion/hash': 0.8.0
'@emotion/memoize': 0.7.4
'@emotion/serialize': 0.11.16
@@ -1888,7 +1704,7 @@ packages:
/babel-plugin-macros@2.8.0:
resolution: {integrity: sha512-SEP5kJpfGYqYKpBrj5XU3ahw5p5GOHJ0U5ssOSQ/WBVdwkD2Dzlce95exQTs3jOVWPPKLBN2rlEWkCK7dSmLvg==}
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
cosmiconfig: 6.0.0
resolve: 1.22.8
dev: false
@@ -1914,8 +1730,8 @@ packages:
tweetnacl: 0.14.5
dev: true
- /body-parser@1.20.2:
- resolution: {integrity: sha512-ml9pReCu3M61kGlqoTm2umSXTlRTuGTx0bfYj+uIUKKYycG5NtSbeetV3faSU6R7ajOPw0g/J1PvK4qNy7s5bA==}
+ /body-parser@1.20.3:
+ resolution: {integrity: sha512-7rAxByjUMqQ3/bHJy7D6OGXvx/MMc4IqBn/X0fcM1QUcAItpZrBEYhWGem+tzXH90c+G01ypMcYJBO9Y30203g==}
engines: {node: '>= 0.8', npm: 1.2.8000 || >= 1.4.16}
dependencies:
bytes: 3.1.2
@@ -1926,7 +1742,7 @@ packages:
http-errors: 2.0.0
iconv-lite: 0.4.24
on-finished: 2.4.1
- qs: 6.11.0
+ qs: 6.13.0
raw-body: 2.5.2
type-is: 1.6.18
unpipe: 1.0.0
@@ -1968,17 +1784,6 @@ packages:
fill-range: 7.1.1
dev: true
- /browserslist@4.23.2:
- resolution: {integrity: sha512-qkqSyistMYdxAcw+CzbZwlBy8AGmS/eEWs+sEV5TnLRGDOL+C5M2EnH6tlZyg0YoAxGJAFKh61En9BR941GnHA==}
- engines: {node: ^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7}
- hasBin: true
- dependencies:
- caniuse-lite: 1.0.30001641
- electron-to-chromium: 1.4.825
- node-releases: 2.0.14
- update-browserslist-db: 1.1.0(browserslist@4.23.2)
- dev: true
-
/bytes@3.0.0:
resolution: {integrity: sha512-pMhOfFDPiv9t5jjIXkHosWmkSyQbvsgEVNkz0ERHbuLh2T/7j4Mqqpz523Fe8MVY89KC6Sh/QfS2sM+SjgFDcw==}
engines: {node: '>= 0.8'}
@@ -2027,21 +1832,12 @@ packages:
engines: {node: '>=6'}
dev: true
- /camelcase@6.3.0:
- resolution: {integrity: sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==}
- engines: {node: '>=10'}
- dev: true
-
- /caniuse-lite@1.0.30001641:
- resolution: {integrity: sha512-Phv5thgl67bHYo1TtMY/MurjkHhV4EDaCosezRXgZ8jzA/Ub+wjxAvbGvjoFENStinwi5kCyOYV3mi5tOGykwA==}
- dev: true
-
/caseless@0.12.0:
resolution: {integrity: sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==}
dev: true
- /chai@4.4.1:
- resolution: {integrity: sha512-13sOfMv2+DWduEU+/xbun3LScLoqN17nBeTLUsmDfKdoiC1fr0n9PU4guu4AhRcOVFk/sW8LyZWHuhWtQZiF+g==}
+ /chai@4.5.0:
+ resolution: {integrity: sha512-RITGBfijLkBddZvnn8jdqoTypxvqbOLYQkGGxXzeFjVHvudaPw0HNFD9x928/eUwYWd2dPCugVqspGALTZZQKw==}
engines: {node: '>=4'}
dependencies:
assertion-error: 1.1.0
@@ -2050,7 +1846,7 @@ packages:
get-func-name: 2.0.2
loupe: 2.3.7
pathval: 1.1.1
- type-detect: 4.0.8
+ type-detect: 4.1.0
dev: true
/chalk@2.4.2:
@@ -2060,6 +1856,7 @@ packages:
ansi-styles: 3.2.1
escape-string-regexp: 1.0.5
supports-color: 5.5.0
+ dev: true
/chalk@4.1.2:
resolution: {integrity: sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==}
@@ -2106,6 +1903,7 @@ packages:
resolution: {integrity: sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==}
dependencies:
color-name: 1.1.3
+ dev: true
/color-convert@2.0.1:
resolution: {integrity: sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==}
@@ -2116,6 +1914,7 @@ packages:
/color-name@1.1.3:
resolution: {integrity: sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==}
+ dev: true
/color-name@1.1.4:
resolution: {integrity: sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==}
@@ -2136,7 +1935,7 @@ packages:
resolution: {integrity: sha512-AF3r7P5dWxL8MxyITRMlORQNaOA2IkAFaTr4k7BUumjPtRpGDTZpl0Pb1XCO6JeDCBdp126Cgs9sMxqSjgYyRg==}
engines: {node: '>= 0.6'}
dependencies:
- mime-db: 1.52.0
+ mime-db: 1.53.0
dev: true
/compression@1.7.4:
@@ -2162,8 +1961,8 @@ packages:
resolution: {integrity: sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==}
dev: true
- /confbox@0.1.7:
- resolution: {integrity: sha512-uJcB/FKZtBMCJpK8MQji6bJHgu1tixKPxRLeGkNzBoOZzpnZUJm0jm2/sBDWcuBx1dYgxV4JU+g5hmNxCyAmdA==}
+ /confbox@0.1.8:
+ resolution: {integrity: sha512-RMtmw0iFkeR4YV+fUOSucriAQNb9g8zFR52MWCtl+cCZOFRNL6zeB395vPzFhEjjn4fMxXudmELnl/KF/WrK6w==}
dev: true
/configstore@4.0.0:
@@ -2198,16 +1997,12 @@ packages:
resolution: {integrity: sha512-ASFBup0Mz1uyiIjANan1jzLQami9z1PoYSZCiiYW2FczPbenXc45FZdBZLzOT+r6+iciuEModtmCti+hjaAk0A==}
dev: false
- /convert-source-map@2.0.0:
- resolution: {integrity: sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg==}
- dev: true
-
/cookie-signature@1.0.6:
resolution: {integrity: sha512-QADzlaHc8icV8I7vbaJXJwod9HWYp8uCqf1xa4OfNu1T7JVxQIrUgOWtHdNDtPiywmFbiS12VjotIXLrKM3orQ==}
dev: true
- /cookie@0.6.0:
- resolution: {integrity: sha512-U71cyTamuh1CRNCfpGY6to28lxvNwPG4Guz/EVjgf3Jmzv0vlDp1atT9eS5dDjMYHucpHbWns6Lwf3BKz6svdw==}
+ /cookie@0.7.1:
+ resolution: {integrity: sha512-6DnInpx7SJ2AK3+CTUE/ZM0vWTUboZCegxhC2xiIydHR9jNuTAASBrfEpHhiGOZw/nX51bHt6YQl8jsGo4y/0w==}
engines: {node: '>= 0.6'}
dev: true
@@ -2245,22 +2040,6 @@ packages:
yaml: 1.10.2
dev: false
- /cosmiconfig@8.3.6(typescript@4.9.5):
- resolution: {integrity: sha512-kcZ6+W5QzcJ3P1Mt+83OUv/oHFqZHIx8DuxG6eZ5RGMERoLqp4BuGjhHLYGK+Kf5XVkQvqBSmAy/nGWN3qDgEA==}
- engines: {node: '>=14'}
- peerDependencies:
- typescript: '>=4.9.5'
- peerDependenciesMeta:
- typescript:
- optional: true
- dependencies:
- import-fresh: 3.3.0
- js-yaml: 4.1.0
- parse-json: 5.2.0
- path-type: 4.0.0
- typescript: 4.9.5
- dev: true
-
/cross-spawn@5.1.0:
resolution: {integrity: sha512-pTgQJ5KC0d2hcY8eyL1IzlBPYjTkyH72XRZPnLyKus2mBfNjQs3klqbJU2VILqZryAZUt9JOb3h/mWMy23/f5A==}
dependencies:
@@ -2339,11 +2118,11 @@ packages:
resolution: {integrity: sha512-fnULvOpxnC5/Vg3NCiWelDsLiUc9bRwAPs/+LfTLNvetFCtCTN+yQz15C/fs4AwX1R9K5GLtLfn8QW+dWisaAw==}
engines: {node: '>=0.11'}
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
dev: false
- /dayjs@1.11.11:
- resolution: {integrity: sha512-okzr3f11N6WuqYtZSvm+F776mB41wRZMhKP+hc34YdW+KmtYYK9iqvHSwo2k9FEH3fhGXvOPV6yz2IcSrfRUDg==}
+ /dayjs@1.11.13:
+ resolution: {integrity: sha512-oaMBel6gjolK862uaPQOVTA7q3TZhuSvuMQAAglQDOWYO9A91IrAOUJEyKVlqJlHE0vq5p5UXxzdPfMH/x6xNg==}
dev: false
/debug@2.6.9:
@@ -2368,8 +2147,8 @@ packages:
ms: 2.0.0
dev: true
- /debug@4.3.5:
- resolution: {integrity: sha512-pt0bNEmneDIvdL1Xsd9oDQ/wrQRkXDT4AUWlNZNPKvW5x/jyO9VFXkJUP07vQ2upmw5PlaITaPKc31jK13V+jg==}
+ /debug@4.3.7:
+ resolution: {integrity: sha512-Er2nc/H7RrMXZBFCEim6TCmMk02Z8vLC2Rbi1KEBggpo0fS6l0S1nnapwmIi3yW/+GOJap1Krg4w0Hg80oCqgQ==}
engines: {node: '>=6.0'}
peerDependencies:
supports-color: '*'
@@ -2377,7 +2156,7 @@ packages:
supports-color:
optional: true
dependencies:
- ms: 2.1.2
+ ms: 2.1.3
/decamelize@1.2.0:
resolution: {integrity: sha512-z2S+W9X73hAUUki+N+9Za2lBlun89zigOyGrsax+KUQ6wKW4ZoWpEYBkGhQjwAjjDCkWxhY0VKEhk8wzY7F5cA==}
@@ -2395,7 +2174,7 @@ packages:
resolution: {integrity: sha512-SUwdGfqdKOwxCPeVYjwSyRpJ7Z+fhpwIAtmCUdZIWZ/YP5R9WAsyuSgpLVDi9bjWoN2LXHNss/dk3urXtdQxGg==}
engines: {node: '>=6'}
dependencies:
- type-detect: 4.0.8
+ type-detect: 4.1.0
dev: true
/deep-extend@0.6.0:
@@ -2469,17 +2248,10 @@ packages:
/dom-helpers@5.2.1:
resolution: {integrity: sha512-nRCa7CK3VTrM2NmGkIy4cbK7IZlgBE/PYMn55rrXefr5xXDP0LdtfPnblFDoVdcAfslJ7or6iqAUnx0CCGIWQA==}
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
csstype: 3.1.3
dev: false
- /dot-case@3.0.4:
- resolution: {integrity: sha512-Kv5nKlh6yRrdrGvxeJ2e5y2eRUpkUosIW4A2AS38zwSz27zu7ufDwQPi5Jhs3XAlGNetl3bmnGhQsMtkKJnj3w==}
- dependencies:
- no-case: 3.0.4
- tslib: 2.6.3
- dev: true
-
/dot-prop@4.2.1:
resolution: {integrity: sha512-l0p4+mIuJIua0mhxGoh4a+iNL9bmeK5DvnSVQa6T0OhrVmaEa1XScX5Etc673FePCJOArq/4Pa2cLGODUWTPOQ==}
engines: {node: '>=4'}
@@ -2513,10 +2285,6 @@ packages:
resolution: {integrity: sha512-WMwm9LhRUo+WUaRN+vRuETqG89IgZphVSNkdFgeb6sS/E4OrDIN7t48CAewSHXc6C8lefD8KKfr5vY61brQlow==}
dev: true
- /electron-to-chromium@1.4.825:
- resolution: {integrity: sha512-OCcF+LwdgFGcsYPYC5keEEFC2XT0gBhrYbeGzHCx7i9qRFbzO/AqTmc/C/1xNhJj+JA7rzlN7mpBuStshh96Cg==}
- dev: true
-
/emoji-regex@7.0.3:
resolution: {integrity: sha512-CwBLREIQ7LvYFB0WyRvwhq5N5qPhc6PMjD6bYggFlI5YyDgl+0vxq5VHbMOFqLg7hfWzmu8T5Z1QofhmTIhItA==}
dev: true
@@ -2534,6 +2302,11 @@ packages:
engines: {node: '>= 0.8'}
dev: true
+ /encodeurl@2.0.0:
+ resolution: {integrity: sha512-Q0n9HRi4m6JuGIV1eFlmvJB7ZEVxu93IrMyiMsGC0lrMJMWzRgx6WGquyfQgZVb31vhGgXnfmPNNXmxnOkRBrg==}
+ engines: {node: '>= 0.8'}
+ dev: true
+
/end-of-stream@1.4.4:
resolution: {integrity: sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==}
dependencies:
@@ -2548,11 +2321,6 @@ packages:
strip-ansi: 6.0.1
dev: true
- /entities@4.5.0:
- resolution: {integrity: sha512-V0hjH4dGPh9Ao5p0MoRY6BVqtwCjhz6vI5LT8AJ55H+4g9/4vbHx1I54fS0XuclLhDHArPQCiMjDxjaL8fPxhw==}
- engines: {node: '>=0.12'}
- dev: true
-
/errno@0.1.8:
resolution: {integrity: sha512-dJ6oBr5SQ1VSd9qkk7ByRgb/1SH4JZjCHSW/mr63/QcXO9zLVxvJ6Oy13nio03rxpSnVDDjFor75SjVeZWPW/A==}
hasBin: true
@@ -2612,7 +2380,7 @@ packages:
object-inspect: 1.13.2
object-keys: 1.1.1
object.assign: 4.1.5
- regexp.prototype.flags: 1.5.2
+ regexp.prototype.flags: 1.5.3
safe-array-concat: 1.1.2
safe-regex-test: 1.0.3
string.prototype.trim: 1.2.9
@@ -2724,11 +2492,6 @@ packages:
'@esbuild/win32-x64': 0.21.5
dev: true
- /escalade@3.1.2:
- resolution: {integrity: sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA==}
- engines: {node: '>=6'}
- dev: true
-
/escape-html@1.0.3:
resolution: {integrity: sha512-NiSupZ4OeuGwr68lGIeym/ksIZMJodUGOSCZ/FSnTxcrekbvqrgdUxlJOMpijaKZVjAJrWrGs/6Jy8OMuyj9ow==}
dev: true
@@ -2801,6 +2564,7 @@ packages:
/eslint@7.32.0:
resolution: {integrity: sha512-VHZ8gX+EDfz+97jGcgyGCyRia/dPOd6Xh9yPv8Bl1+SoaIwD+a/vlrOmGRUyOYu7MwUhc7CxqeaDZU13S4+EpA==}
engines: {node: ^10.12.0 || >=12.0.0}
+ deprecated: This version is no longer supported. Please see https://eslint.org/version-support for other options.
hasBin: true
dependencies:
'@babel/code-frame': 7.12.11
@@ -2809,7 +2573,7 @@ packages:
ajv: 6.12.6
chalk: 4.1.2
cross-spawn: 7.0.3
- debug: 4.3.5
+ debug: 4.3.7
doctrine: 3.0.0
enquirer: 2.4.1
escape-string-regexp: 4.0.0
@@ -2837,7 +2601,7 @@ packages:
optionator: 0.9.4
progress: 2.0.3
regexpp: 3.2.0
- semver: 7.6.2
+ semver: 7.6.3
strip-ansi: 6.0.1
strip-json-comments: 3.1.1
table: 6.8.2
@@ -2886,14 +2650,10 @@ packages:
engines: {node: '>=4.0'}
dev: true
- /estree-walker@2.0.2:
- resolution: {integrity: sha512-Rfkk/Mp/DL7JVje3u18FxFujQlTNR2q6QfMSMB7AvCBx91NGj/ba3kCfza0f6dVDbw7YlRf/nDrn7pQrCCyQ/w==}
- dev: true
-
/estree-walker@3.0.3:
resolution: {integrity: sha512-7RUKfXgSMMkzt6ZuXmqapOurLGPPfgj6l9uRZ7lRGolvk0y2yocc35LdcxKC5PQZdn2DMqioAQ2NoWcrTKmm6g==}
dependencies:
- '@types/estree': 1.0.5
+ '@types/estree': 1.0.6
dev: true
/esutils@2.0.3:
@@ -2937,42 +2697,42 @@ packages:
/express-urlrewrite@1.4.0:
resolution: {integrity: sha512-PI5h8JuzoweS26vFizwQl6UTF25CAHSggNv0J25Dn/IKZscJHWZzPrI5z2Y2jgOzIaw2qh8l6+/jUcig23Z2SA==}
dependencies:
- debug: 4.3.5
- path-to-regexp: 1.8.0
+ debug: 4.3.7
+ path-to-regexp: 1.9.0
transitivePeerDependencies:
- supports-color
dev: true
- /express@4.19.2:
- resolution: {integrity: sha512-5T6nhjsT+EOMzuck8JjBHARTHfMht0POzlA60WV2pMD3gyXw2LZnZ+ueGdNxG+0calOJcWKbpFcuzLZ91YWq9Q==}
+ /express@4.21.1:
+ resolution: {integrity: sha512-YSFlK1Ee0/GC8QaO91tHcDxJiE/X4FbpAyQWkxAvG6AXCuR65YzK8ua6D9hvi/TzUfZMpc+BwuM1IPw8fmQBiQ==}
engines: {node: '>= 0.10.0'}
dependencies:
accepts: 1.3.8
array-flatten: 1.1.1
- body-parser: 1.20.2
+ body-parser: 1.20.3
content-disposition: 0.5.4
content-type: 1.0.5
- cookie: 0.6.0
+ cookie: 0.7.1
cookie-signature: 1.0.6
debug: 2.6.9
depd: 2.0.0
- encodeurl: 1.0.2
+ encodeurl: 2.0.0
escape-html: 1.0.3
etag: 1.8.1
- finalhandler: 1.2.0
+ finalhandler: 1.3.1
fresh: 0.5.2
http-errors: 2.0.0
- merge-descriptors: 1.0.1
+ merge-descriptors: 1.0.3
methods: 1.1.2
on-finished: 2.4.1
parseurl: 1.3.3
- path-to-regexp: 0.1.7
+ path-to-regexp: 0.1.10
proxy-addr: 2.0.7
- qs: 6.11.0
+ qs: 6.13.0
range-parser: 1.2.1
safe-buffer: 5.2.1
- send: 0.18.0
- serve-static: 1.15.0
+ send: 0.19.0
+ serve-static: 1.16.2
setprototypeof: 1.2.0
statuses: 2.0.1
type-is: 1.6.18
@@ -3007,7 +2767,7 @@ packages:
'@nodelib/fs.walk': 1.2.8
glob-parent: 5.1.2
merge2: 1.4.1
- micromatch: 4.0.7
+ micromatch: 4.0.8
dev: true
/fast-json-stable-stringify@2.1.0:
@@ -3018,6 +2778,10 @@ packages:
resolution: {integrity: sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw==}
dev: true
+ /fast-uri@3.0.3:
+ resolution: {integrity: sha512-aLrHthzCjH5He4Z2H9YZ+v6Ujb9ocRuW6ZzkJQOrTxleEijANq4v1TsaPaVG1PZcuurEzrLcWRyYBYXD5cEiaw==}
+ dev: true
+
/fastq@1.17.1:
resolution: {integrity: sha512-sRVD3lWVIXWg6By68ZN7vho9a1pQcN/WBFaAAsDDFzlJjvoGx0P8z7V1t72grFJfJhu3YPZBuu25f7Kaw2jN1w==}
dependencies:
@@ -3043,12 +2807,12 @@ packages:
to-regex-range: 5.0.1
dev: true
- /finalhandler@1.2.0:
- resolution: {integrity: sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg==}
+ /finalhandler@1.3.1:
+ resolution: {integrity: sha512-6BN9trH7bp3qvnrRyzsBz+g3lZxTNZTbVO2EV1CS0WIcDbawYVdYvGflME/9QP0h0pYlCDBCTjYa9nZzMDpyxQ==}
engines: {node: '>= 0.8'}
dependencies:
debug: 2.6.9
- encodeurl: 1.0.2
+ encodeurl: 2.0.0
escape-html: 1.0.3
on-finished: 2.4.1
parseurl: 1.3.3
@@ -3082,8 +2846,8 @@ packages:
resolution: {integrity: sha512-X8cqMLLie7KsNUDSdzeN8FYK9rEt4Dt67OsG/DNGnYTSDBG4uFAJFBnUeiV+zCVAvwFy56IjM9sH51jVaEhNxw==}
dev: true
- /follow-redirects@1.15.6:
- resolution: {integrity: sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA==}
+ /follow-redirects@1.15.9:
+ resolution: {integrity: sha512-gew4GsXizNgdoRyqmyfMHyAmXsZDk6mHkSxZFCzW9gwlbtOW44CDtYavM+y+72qD/Vq2l550kMF52DT8fOLJqQ==}
engines: {node: '>=4.0'}
peerDependencies:
debug: '*'
@@ -3098,8 +2862,8 @@ packages:
is-callable: 1.2.7
dev: true
- /foreground-child@3.2.1:
- resolution: {integrity: sha512-PXUUyLqrR2XCWICfv6ukppP96sdFwWbNEnfEMt7jNsISjMsvaLNinAHNDYyvkyU+SZG2BTSbT5NjG+vZslfGTA==}
+ /foreground-child@3.3.0:
+ resolution: {integrity: sha512-Ld2g8rrAyMYFXBhEqMz8ZAHBi4J4uS1i/CxGMDnjyFWddMXLVcDp051DZfu+t7+ab7Wv6SMqpWmyFIj5UbfFvg==}
engines: {node: '>=14'}
dependencies:
cross-spawn: 7.0.3
@@ -3119,8 +2883,8 @@ packages:
mime-types: 2.1.35
dev: true
- /form-data@4.0.0:
- resolution: {integrity: sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==}
+ /form-data@4.0.1:
+ resolution: {integrity: sha512-tzN8e4TX8+kkxGPK8D5u0FNmjPUjw3lwC9lSLxxoB/+GtsJG91CO8bSWy73APlgAZzZbXEYZJuxjkHH2w+Ezhw==}
engines: {node: '>= 6'}
dependencies:
asynckit: 0.4.0
@@ -3171,11 +2935,6 @@ packages:
resolution: {integrity: sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==}
dev: true
- /gensync@1.0.0-beta.2:
- resolution: {integrity: sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg==}
- engines: {node: '>=6.9.0'}
- dev: true
-
/get-caller-file@2.0.5:
resolution: {integrity: sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==}
engines: {node: 6.* || 8.* || >= 10.*}
@@ -3205,14 +2964,14 @@ packages:
resolution: {integrity: sha512-GMat4EJ5161kIy2HevLlr4luNjBgvmj413KaQA7jt4V8B4RDsfpHk7WQ9GVqfYyyx8OS/L66Kox+rJRNklLK7w==}
engines: {node: '>=6'}
dependencies:
- pump: 3.0.0
+ pump: 3.0.2
dev: true
/get-stream@5.2.0:
resolution: {integrity: sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==}
engines: {node: '>=8'}
dependencies:
- pump: 3.0.0
+ pump: 3.0.2
dev: true
/get-stream@8.0.1:
@@ -3250,11 +3009,11 @@ packages:
resolution: {integrity: sha512-7Bv8RF0k6xjo7d4A/PxYLbUCfb6c+Vpd2/mB2yRDlew7Jb5hEXiCD9ibfO7wpk8i4sevK6DFny9h7EYbM3/sHg==}
hasBin: true
dependencies:
- foreground-child: 3.2.1
+ foreground-child: 3.3.0
jackspeak: 3.4.3
minimatch: 9.0.5
minipass: 7.1.2
- package-json-from-dist: 1.0.0
+ package-json-from-dist: 1.0.1
path-scurry: 1.11.1
dev: true
@@ -3280,6 +3039,7 @@ packages:
/globals@11.12.0:
resolution: {integrity: sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==}
engines: {node: '>=4'}
+ dev: false
/globals@13.24.0:
resolution: {integrity: sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==}
@@ -3303,7 +3063,7 @@ packages:
array-union: 2.1.0
dir-glob: 3.0.1
fast-glob: 3.3.2
- ignore: 5.3.1
+ ignore: 5.3.2
merge2: 1.4.1
slash: 3.0.0
dev: true
@@ -3365,6 +3125,7 @@ packages:
/has-flag@3.0.0:
resolution: {integrity: sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw==}
engines: {node: '>=4'}
+ dev: true
/has-flag@4.0.0:
resolution: {integrity: sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==}
@@ -3408,7 +3169,7 @@ packages:
/history@4.10.1:
resolution: {integrity: sha512-36nwAD620w12kuzPAsyINPWJqlNbij+hpK1k9XRloDtym8mxzGYl2c17LnV6IAGB2Dmg4tEa7G7DlawS0+qjew==}
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
loose-envify: 1.4.0
resolve-pathname: 3.0.0
tiny-invariant: 1.3.3
@@ -3419,7 +3180,7 @@ packages:
/history@5.3.0:
resolution: {integrity: sha512-ZqaKwjjrAYUYfLG+htGaIIZ4nioX2L70ZUMIFysS3xvBsSG4x/n1V6TXV3N8ZYNuFGlDirFg32T7B6WOUPDYcQ==}
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
dev: true
/hoist-non-react-statics@3.3.2:
@@ -3473,8 +3234,8 @@ packages:
engines: {node: '>= 4'}
dev: true
- /ignore@5.3.1:
- resolution: {integrity: sha512-5Fytz/IraMjqpwfd34ke28PTVMjZjJG2MPn5t7OE4eUCUNf8BAa7b5WUS9/Qvr6mwOQS7Mk6vdsMno5he+T8Xw==}
+ /ignore@5.3.2:
+ resolution: {integrity: sha512-hsBTNUqQTDwkWtcdYI2i06Y/nUBEsNEDJKjWdigLvegy8kDuJAS8uRlpkkcQpyEXL0Z/pjDy5HBmMjRCJ2gq+g==}
engines: {node: '>= 4'}
dev: true
@@ -3569,8 +3330,8 @@ packages:
ci-info: 2.0.0
dev: true
- /is-core-module@2.14.0:
- resolution: {integrity: sha512-a5dFJih5ZLYlRtDc0dZWP7RiKr6xIKzmn/oAYCDvdLThadVgyJwlaoQPmRtMSpz+rk0OGAgIu+TcM9HUF0fk1A==}
+ /is-core-module@2.15.1:
+ resolution: {integrity: sha512-z0vtXSwucUJtANQWldhbtbt7BnL0vxiFjIdDLAatwhDYty2bad6s+rijD6Ri4YuYJubLzIJLUidCh09e1djEVQ==}
engines: {node: '>= 0.4'}
dependencies:
hasown: 2.0.2
@@ -3762,21 +3523,15 @@ packages:
esprima: 4.0.1
dev: true
- /js-yaml@4.1.0:
- resolution: {integrity: sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==}
- hasBin: true
- dependencies:
- argparse: 2.0.1
- dev: true
-
/jsbn@0.1.1:
resolution: {integrity: sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==}
dev: true
- /jsesc@2.5.2:
- resolution: {integrity: sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==}
- engines: {node: '>=4'}
+ /jsesc@3.0.2:
+ resolution: {integrity: sha512-xKqzzWXDttJuOcawBt4KnKHHIf5oQ/Cxax+0PWFG+DFDgHNAdi+TXECADI+RYiFUMmx8792xsMbbgXj4CwnP4g==}
+ engines: {node: '>=6'}
hasBin: true
+ dev: false
/json-buffer@3.0.0:
resolution: {integrity: sha512-CuUqjv0FUZIdXkHPI8MezCnFCdaTAacej1TZYulLoAg1h/PhwkdXFN4V/gzY4g+fMBCOV2xF+rp7t2XD2ns/NQ==}
@@ -3792,6 +3547,7 @@ packages:
/json-parse-even-better-errors@2.3.1:
resolution: {integrity: sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==}
+ dev: false
/json-parse-helpfulerror@1.0.3:
resolution: {integrity: sha512-XgP0FGR77+QhUxjXkwOMkC94k3WtqEBfcnjWqhRd82qTat4SWKRE+9kUnynz/shm3I4ea2+qISvTIeGTNU7kJg==}
@@ -3816,13 +3572,13 @@ packages:
engines: {node: '>=8'}
hasBin: true
dependencies:
- body-parser: 1.20.2
+ body-parser: 1.20.3
chalk: 2.4.2
compression: 1.7.4
connect-pause: 0.1.1
cors: 2.8.5
errorhandler: 1.5.1
- express: 4.19.2
+ express: 4.21.1
express-urlrewrite: 1.4.0
json-parse-helpfulerror: 1.0.3
lodash: 4.17.21
@@ -3904,7 +3660,7 @@ packages:
image-size: 0.5.5
make-dir: 2.1.0
mime: 1.6.0
- native-request: 1.1.0
+ native-request: 1.1.2
source-map: 0.6.1
/levn@0.4.1:
@@ -3932,8 +3688,8 @@ packages:
resolution: {integrity: sha512-ok6z3qlYyCDS4ZEU27HaU6x/xZa9Whf8jD4ptH5UZTQYZVYeb9bnZ3ojVhiJNLiXK1Hfc0GNbLXcmZ5plLDDBg==}
engines: {node: '>=14'}
dependencies:
- mlly: 1.7.1
- pkg-types: 1.1.3
+ mlly: 1.7.2
+ pkg-types: 1.2.1
dev: true
/locate-path@3.0.0:
@@ -3984,12 +3740,6 @@ packages:
steno: 0.4.4
dev: true
- /lower-case@2.0.2:
- resolution: {integrity: sha512-7fm3l3NAF9WfN6W3JOmf5drwpVqX78JtoGJ3A6W0a6ZnldM41w2fV5D490psKFTpMds8TJse/eHLFFsNHHjHgg==}
- dependencies:
- tslib: 2.6.3
- dev: true
-
/lowercase-keys@1.0.1:
resolution: {integrity: sha512-G2Lj61tXDnVFFOi8VZds+SoQjtQC3dgokKdDG2mTm1tx4m50NUHBOZSBwQQHyy0V12A0JTG4icfZQH+xPyh8VA==}
engines: {node: '>=0.10.0'}
@@ -4011,14 +3761,8 @@ packages:
yallist: 2.1.2
dev: true
- /lru-cache@5.1.1:
- resolution: {integrity: sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==}
- dependencies:
- yallist: 3.1.1
- dev: true
-
- /magic-string@0.30.10:
- resolution: {integrity: sha512-iIRwTIf0QKV3UAnYK4PU8uiEc4SRh5jX0mwpIwETPpHdhVM4f53RSwS/vXvN1JhGX+Cs7B8qIq3d6AH49O5fAQ==}
+ /magic-string@0.30.12:
+ resolution: {integrity: sha512-Ea8I3sQMVXr8JhN4z+H/d8zwo+tYDgHE9+5G4Wnrwhs0gaK9fXTKx0Tw5Xwsd/bCPTTZNRAdpyzvoeORe9LYpw==}
dependencies:
'@jridgewell/sourcemap-codec': 1.5.0
dev: true
@@ -4053,8 +3797,8 @@ packages:
engines: {node: '>= 0.10.0'}
dev: true
- /merge-descriptors@1.0.1:
- resolution: {integrity: sha512-cCi6g3/Zr1iqQi6ySbseM1Xvooa98N0w31jzUYrXPX2xqObmFGHJ0tQ5u74H3mVh7wLouTseZyYIq39g8cNp1w==}
+ /merge-descriptors@1.0.3:
+ resolution: {integrity: sha512-gaNvAS7TZ897/rVaZ0nMtAyxNyi/pdbjbAwUpFQpN70GqnVfOiXpeUUMKRBmzXaSQ8DdTX4/0ms62r2K+hE6mQ==}
dev: true
/merge-stream@2.0.0:
@@ -4083,8 +3827,8 @@ packages:
engines: {node: '>= 0.6'}
dev: true
- /micromatch@4.0.7:
- resolution: {integrity: sha512-LPP/3KorzCwBxfeUuZmaR6bG2kdeHSbe0P2tY3FLRU4vYrjYz5hI4QZwV0njUx3jeuKe67YukQ1LSPZBKDqO/Q==}
+ /micromatch@4.0.8:
+ resolution: {integrity: sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==}
engines: {node: '>=8.6'}
dependencies:
braces: 3.0.3
@@ -4095,6 +3839,11 @@ packages:
resolution: {integrity: sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==}
engines: {node: '>= 0.6'}
+ /mime-db@1.53.0:
+ resolution: {integrity: sha512-oHlN/w+3MQ3rba9rqFr6V/ypF10LSkdwUysQL7GkXoTgIWeV+tcXGA852TBxH+gsh8UWoyhR1hKcoMJTuWflpg==}
+ engines: {node: '>= 0.6'}
+ dev: true
+
/mime-types@2.1.35:
resolution: {integrity: sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==}
engines: {node: '>= 0.6'}
@@ -4150,13 +3899,13 @@ packages:
engines: {node: '>=16 || 14 >=14.17'}
dev: true
- /mlly@1.7.1:
- resolution: {integrity: sha512-rrVRZRELyQzrIUAVMHxP97kv+G786pHmOKzuFII8zDYahFBS7qnHh2AlYSl1GAHhaMPCz6/oHjVMcfFYgFYHgA==}
+ /mlly@1.7.2:
+ resolution: {integrity: sha512-tN3dvVHYVz4DhSXinXIk7u9syPYaJvio118uomkovAtWBT+RdbP6Lfh/5Lvo519YMmwBafwlh20IPTXIStscpA==}
dependencies:
- acorn: 8.12.1
+ acorn: 8.13.0
pathe: 1.1.2
- pkg-types: 1.1.3
- ufo: 1.5.3
+ pkg-types: 1.2.1
+ ufo: 1.5.4
dev: true
/moment@2.30.1:
@@ -4180,12 +3929,8 @@ packages:
resolution: {integrity: sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==}
dev: true
- /ms@2.1.2:
- resolution: {integrity: sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==}
-
/ms@2.1.3:
resolution: {integrity: sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==}
- dev: true
/mz@2.7.0:
resolution: {integrity: sha512-z81GNO7nnYMEhrGh9LeymoE4+Yr0Wn5McHIZMK5cfQCl+NDX08sCZgUc9/6MHni9IWuFLm1Z3HTCXu2z9fN62Q==}
@@ -4205,8 +3950,8 @@ packages:
hasBin: true
dev: true
- /native-request@1.1.0:
- resolution: {integrity: sha512-uZ5rQaeRn15XmpgE0xoPL8YWqcX90VtCFglYwAgkvKM5e8fog+vePLAhHxuuv/gRkrQxIeh5U3q9sMNUrENqWw==}
+ /native-request@1.1.2:
+ resolution: {integrity: sha512-/etjwrK0J4Ebbcnt35VMWnfiUX/B04uwGJxyJInagxDqf2z5drSt/lsOvEMWGYunz1kaLZAFrV4NDAbOoDKvAQ==}
requiresBuild: true
optional: true
@@ -4227,17 +3972,6 @@ packages:
resolution: {integrity: sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ==}
dev: true
- /no-case@3.0.4:
- resolution: {integrity: sha512-fgAN3jGAh+RoxUGZHTSOLJIqUc2wmoBwGR4tbpNAKmmovFoWq0OdRkb0VkldReO2a2iBT/OEulG9XSUc10r3zg==}
- dependencies:
- lower-case: 2.0.2
- tslib: 2.6.3
- dev: true
-
- /node-releases@2.0.14:
- resolution: {integrity: sha512-y10wOWt8yZpqXmOgRo77WaHEmhYQYGNA6y421PKsKYWEK8aW+cqAphborZDhqfyKrbZEN92CN1X2KbafY2s7Yw==}
- dev: true
-
/normalize-package-data@2.5.0:
resolution: {integrity: sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA==}
dependencies:
@@ -4390,8 +4124,8 @@ packages:
engines: {node: '>=6'}
dev: true
- /package-json-from-dist@1.0.0:
- resolution: {integrity: sha512-dATvCeZN/8wQsGywez1mzHtTlP22H8OEfPrVMLNr4/eGa+ijtLn/6M5f0dY8UKNrC2O9UCU6SSoG3qRKnt7STw==}
+ /package-json-from-dist@1.0.1:
+ resolution: {integrity: sha512-UEZIS3/by4OC8vL3P2dTXRETpebLI2NiI5vIrjaD/5UtrkFX/tNbwjTSRAGC/+7CAo2pIcBaRgWmcBBHcsaCIw==}
dev: true
/package-json@6.5.0:
@@ -4422,10 +4156,11 @@ packages:
resolution: {integrity: sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg==}
engines: {node: '>=8'}
dependencies:
- '@babel/code-frame': 7.24.7
+ '@babel/code-frame': 7.26.0
error-ex: 1.3.2
json-parse-even-better-errors: 2.3.1
lines-and-columns: 1.2.4
+ dev: false
/parse-ms@2.1.0:
resolution: {integrity: sha512-kHt7kzLoS9VBZfUsiKjv43mr91ea+U05EyKkEtqp7vNbHxmaVuEqN7XxeEVnGrMtYOAxGrDElSi96K7EgO1zCA==}
@@ -4477,12 +4212,12 @@ packages:
minipass: 7.1.2
dev: true
- /path-to-regexp@0.1.7:
- resolution: {integrity: sha512-5DFkuoqlv1uYQKxy8omFBeJPQcdoE07Kv2sferDCrAq1ohOU+MSDswDIbnx3YAM60qIOnYa53wBhXW0EbMonrQ==}
+ /path-to-regexp@0.1.10:
+ resolution: {integrity: sha512-7lf7qcQidTku0Gu3YDPc8DJ1q7OOucfa/BSsIwjuh56VU7katFvuM8hULfkwB3Fns/rsVF7PwPKVw1sl5KQS9w==}
dev: true
- /path-to-regexp@1.8.0:
- resolution: {integrity: sha512-n43JRhlUKUAlibEJhPeir1ncUID16QnEjNpwzNdO3Lm4ywrBpBZ5oLD0I6br9evr1Y9JTqwRtAh7JLoOzAQdVA==}
+ /path-to-regexp@1.9.0:
+ resolution: {integrity: sha512-xIp7/apCFJuUHdDLWe8O1HIkb0kQrOMb/0u6FXQjemHn/ii5LrIzU6bdECnsiTF/GjZkMEKg1xdiZwNqDYlZ6g==}
dependencies:
isarray: 0.0.1
@@ -4509,8 +4244,8 @@ packages:
resolution: {integrity: sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==}
dev: true
- /picocolors@1.0.1:
- resolution: {integrity: sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew==}
+ /picocolors@1.1.1:
+ resolution: {integrity: sha512-xceH2snhtb5M9liqDsmEw56le376mTZkEX/jEb/RxNFyegNul7eNslCXP9FDj/Lcu0X8KEyMceP2ntpaHrDEVA==}
/picomatch@2.3.1:
resolution: {integrity: sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==}
@@ -4539,11 +4274,11 @@ packages:
engines: {node: '>= 6'}
dev: true
- /pkg-types@1.1.3:
- resolution: {integrity: sha512-+JrgthZG6m3ckicaOB74TwQ+tBWsFl3qVQg7mN8ulwSOElJ7gBhKzj2VkCPnZ4NlF6kEquYU+RIYNVAvzd54UA==}
+ /pkg-types@1.2.1:
+ resolution: {integrity: sha512-sQoqa8alT3nHjGuTjuKgOnvjo4cljkufdtLMnO2LBP/wRwuDlo1tkaEdMxCRhyGRPacv/ztlZgDPm2b7FAmEvw==}
dependencies:
- confbox: 0.1.7
- mlly: 1.7.1
+ confbox: 0.1.8
+ mlly: 1.7.2
pathe: 1.1.2
dev: true
@@ -4563,13 +4298,13 @@ packages:
engines: {node: '>= 0.4'}
dev: true
- /postcss@8.4.39:
- resolution: {integrity: sha512-0vzE+lAiG7hZl1/9I8yzKLx3aR9Xbof3fBHKunvMfOCYAtMhrsnccJY2iTURb9EZd5+pLuiNV9/c/GZJOHsgIw==}
+ /postcss@8.4.47:
+ resolution: {integrity: sha512-56rxCq7G/XfB4EkXq9Egn5GCqugWvDFjafDOThIdMBsI15iqPqR5r15TfSr1YPYeEI19YeaXMCbY6u88Y76GLQ==}
engines: {node: ^10 || ^12 || >=14}
dependencies:
nanoid: 3.3.7
- picocolors: 1.0.1
- source-map-js: 1.2.0
+ picocolors: 1.1.1
+ source-map-js: 1.2.1
dev: true
/prelude-ls@1.2.1:
@@ -4649,8 +4384,8 @@ packages:
resolution: {integrity: sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag==}
dev: true
- /pump@3.0.0:
- resolution: {integrity: sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww==}
+ /pump@3.0.2:
+ resolution: {integrity: sha512-tUPXtzlGM8FE3P0ZL6DVs/3P58k9nk8/jZeQCurTJylQA8qFYzHFfhBJkuqyE0FifOsQ0uKWekiZ5g8wtr28cw==}
dependencies:
end-of-stream: 1.4.4
once: 1.4.0
@@ -4661,8 +4396,8 @@ packages:
engines: {node: '>=6'}
dev: true
- /qs@6.11.0:
- resolution: {integrity: sha512-MvjoMCJwEarSbUYk5O+nmoSzSutSsTwF85zcHPQ9OrlFoZOYIjaqBAJIqIXjptyD5vThxGq52Xu/MaJzRkIk4Q==}
+ /qs@6.13.0:
+ resolution: {integrity: sha512-+38qI9SOr8tfZ4QmJNplMUxqjbe7LKvvZgWdExBOmd+egZTtjLB67Gu0HRX3u/XOq7UU2Nx6nsjvS16Z9uwfpg==}
engines: {node: '>=0.6'}
dependencies:
side-channel: 1.0.6
@@ -4698,7 +4433,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
dom-align: 1.12.4
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -4713,7 +4448,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
array-tree-filter: 2.1.0
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -4728,7 +4463,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4740,9 +4475,9 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4755,9 +4490,9 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4769,7 +4504,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4782,7 +4517,7 @@ packages:
react: '*'
react-dom: '*'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4796,7 +4531,7 @@ packages:
react: '>= 16.9.0'
react-dom: '>= 16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
async-validator: 3.5.2
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4809,7 +4544,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-dialog: 8.5.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -4823,7 +4558,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4836,7 +4571,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-menu: 8.10.8(react-dom@16.14.0)(react@16.14.0)
rc-textarea: 0.3.7(react-dom@16.14.0)(react@16.14.0)
@@ -4852,10 +4587,10 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
mini-store: 3.0.6(react-dom@16.14.0)(react@16.14.0)
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4864,13 +4599,13 @@ packages:
shallowequal: 1.1.0
dev: false
- /rc-motion@2.9.2(react-dom@16.14.0)(react@16.14.0):
- resolution: {integrity: sha512-fUAhHKLDdkAXIDLH0GYwof3raS58dtNUmzLF2MeiR8o6n4thNpSDQhOqQzWE4WfFZDCi9VEN8n7tiB7czREcyw==}
+ /rc-motion@2.9.3(react-dom@16.14.0)(react@16.14.0):
+ resolution: {integrity: sha512-rkW47ABVkic7WEB0EKJqzySpvDqwl60/tdkY7hWP7dYnh5pm0SzJpo54oW3TDUGXV5wfxXFmMkxrzRRbotQ0+w==}
peerDependencies:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4884,9 +4619,9 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4898,7 +4633,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-resize-observer: 1.4.0(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -4912,7 +4647,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4925,10 +4660,10 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
date-fns: 2.30.0
- dayjs: 1.11.11
+ dayjs: 1.11.13
moment: 2.30.1
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -4943,7 +4678,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -4956,7 +4691,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4969,7 +4704,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -4984,13 +4719,13 @@ packages:
react: '*'
react-dom: '*'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-overflow: 1.3.2(react-dom@16.14.0)(react@16.14.0)
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
- rc-virtual-list: 3.14.5(react-dom@16.14.0)(react@16.14.0)
+ rc-virtual-list: 3.14.8(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
dev: false
@@ -5002,7 +4737,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-tooltip: 5.0.2(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -5018,7 +4753,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -5031,7 +4766,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -5045,7 +4780,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-resize-observer: 1.4.0(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -5061,7 +4796,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-dropdown: 3.2.5(react-dom@16.14.0)(react@16.14.0)
rc-menu: 8.10.8(react-dom@16.14.0)(react@16.14.0)
@@ -5077,7 +4812,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-resize-observer: 1.4.0(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -5092,7 +4827,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
rc-trigger: 5.3.4(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -5104,7 +4839,7 @@ packages:
react: '*'
react-dom: '*'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-select: 12.1.13(react-dom@16.14.0)(react@16.14.0)
rc-tree: 4.1.5(react-dom@16.14.0)(react@16.14.0)
@@ -5120,11 +4855,11 @@ packages:
react: '*'
react-dom: '*'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
- rc-virtual-list: 3.14.5(react-dom@16.14.0)(react@16.14.0)
+ rc-virtual-list: 3.14.8(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
dev: false
@@ -5136,10 +4871,10 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-align: 4.0.15(react-dom@16.14.0)(react@16.14.0)
- rc-motion: 2.9.2(react-dom@16.14.0)(react@16.14.0)
+ rc-motion: 2.9.3(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
@@ -5151,7 +4886,7 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
react: 16.14.0
@@ -5164,20 +4899,20 @@ packages:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
react: 16.14.0
react-dom: 16.14.0(react@16.14.0)
react-is: 18.3.1
dev: false
- /rc-virtual-list@3.14.5(react-dom@16.14.0)(react@16.14.0):
- resolution: {integrity: sha512-ZMOnkCLv2wUN8Jz7yI4XiSLa9THlYvf00LuMhb1JlsQCewuU7ydPuHw1rGVPhe9VZYl/5UqODtNd7QKJ2DMGfg==}
+ /rc-virtual-list@3.14.8(react-dom@16.14.0)(react@16.14.0):
+ resolution: {integrity: sha512-8D0KfzpRYi6YZvlOWIxiOm9BGt4Wf2hQyEaM6RXlDDiY2NhLheuYI+RA+7ZaZj1lq+XQqy3KHlaeeXQfzI5fGg==}
engines: {node: '>=8.x'}
peerDependencies:
react: '>=16.9.0'
react-dom: '>=16.9.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
classnames: 2.5.1
rc-resize-observer: 1.4.0(react-dom@16.14.0)(react@16.14.0)
rc-util: 5.43.0(react-dom@16.14.0)(react@16.14.0)
@@ -5223,17 +4958,12 @@ packages:
/react-is@18.3.1:
resolution: {integrity: sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==}
- /react-refresh@0.14.2:
- resolution: {integrity: sha512-jCvmsr+1IUSMUyzOkRcvnVbX3ZYC6g9TDrDbFuFmRDq7PD4yaGbLKNQL6k2jnArV8hjYxh7hVhAZB6s9HDGpZA==}
- engines: {node: '>=0.10.0'}
- dev: true
-
/react-router-dom@5.3.4(react@16.14.0):
resolution: {integrity: sha512-m4EqFMHv/Ih4kpcBCONHbkT68KoAeHN4p3lAGoNryfHi0dMy0kCzEZakiKRsvg5wHZ/JLrLW8o8KomWiz/qbYQ==}
peerDependencies:
react: '>=15'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
history: 4.10.1
loose-envify: 1.4.0
prop-types: 15.8.1
@@ -5248,11 +4978,11 @@ packages:
peerDependencies:
react: '>=15'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
history: 4.10.1
hoist-non-react-statics: 3.3.2
loose-envify: 1.4.0
- path-to-regexp: 1.8.0
+ path-to-regexp: 1.9.0
prop-types: 15.8.1
react: 16.14.0
react-is: 16.13.1
@@ -5266,7 +4996,7 @@ packages:
react: ^16.8.0 || ^17.0.0
react-dom: ^16.8.0 || ^17.0.0
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
'@emotion/cache': 10.0.29
'@emotion/core': 10.3.1(react@16.14.0)
'@emotion/css': 10.0.27
@@ -5286,7 +5016,7 @@ packages:
react: '>=16.6.0'
react-dom: '>=16.6.0'
dependencies:
- '@babel/runtime': 7.24.8
+ '@babel/runtime': 7.26.0
dom-helpers: 5.2.1
loose-envify: 1.4.0
prop-types: 15.8.1
@@ -5325,8 +5055,8 @@ packages:
/regenerator-runtime@0.14.1:
resolution: {integrity: sha512-dYnhHh0nJoMfnkZs6GmmhFknAGRrLznOu5nc9ML+EJxGvrx6H7teuevqVqCuPcPK//3eDrrjQhehXVx9cnkGdw==}
- /regexp.prototype.flags@1.5.2:
- resolution: {integrity: sha512-NcDiDkTLuPR+++OCKB0nWafEmhg/Da8aUPLPMQbK+bxKKCm1/S5he+AqYa4PlMCVBalb4/yxIRub6qkEx5yJbw==}
+ /regexp.prototype.flags@1.5.3:
+ resolution: {integrity: sha512-vqlC04+RQoFalODCbCumG2xIOvapzVMHwsyIGM/SIE8fRhFFsXeH8/QQ+s0T0kDAhKc4k30s73/0ydkHQz6HlQ==}
engines: {node: '>= 0.4'}
dependencies:
call-bind: 1.0.7
@@ -5360,7 +5090,7 @@ packages:
deprecated: request has been deprecated, see https://github.com/request/request/issues/3142
dependencies:
aws-sign2: 0.7.0
- aws4: 1.13.0
+ aws4: 1.13.2
caseless: 0.12.0
combined-stream: 1.0.8
extend: 3.0.2
@@ -5411,7 +5141,7 @@ packages:
resolution: {integrity: sha512-oKWePCxqpd6FlLvGV1VU0x7bkPmmCNolxzjMf4NczoDnQcIWrAF+cPtZn5i6n+RfD2d9i0tzpKnG6Yk168yIyw==}
hasBin: true
dependencies:
- is-core-module: 2.14.0
+ is-core-module: 2.15.1
path-parse: 1.0.7
supports-preserve-symlinks-flag: 1.0.0
@@ -5434,37 +5164,37 @@ packages:
glob: 7.2.3
dev: true
- /rollup@3.29.4:
- resolution: {integrity: sha512-oWzmBZwvYrU0iJHtDmhsm662rC15FRXmcjCk1xD771dFDx5jJ02ufAQQTn0etB2emNk4J9EZg/yWKpsn9BWGRw==}
+ /rollup@3.29.5:
+ resolution: {integrity: sha512-GVsDdsbJzzy4S/v3dqWPJ7EfvZJfCHiDqe80IyrF59LYuP+e6U1LJoUqeuqRbwAWoMNoXivMNeNAOf5E22VA1w==}
engines: {node: '>=14.18.0', npm: '>=8.0.0'}
hasBin: true
optionalDependencies:
fsevents: 2.3.3
dev: true
- /rollup@4.18.1:
- resolution: {integrity: sha512-Elx2UT8lzxxOXMpy5HWQGZqkrQOtrVDDa/bm9l10+U4rQnVzbL/LgZ4NOM1MPIDyHk69W4InuYDF5dzRh4Kw1A==}
+ /rollup@4.24.0:
+ resolution: {integrity: sha512-DOmrlGSXNk1DM0ljiQA+i+o0rSLhtii1je5wgk60j49d1jHT5YYttBv1iWOnYSTG+fZZESUOSNiAl89SIet+Cg==}
engines: {node: '>=18.0.0', npm: '>=8.0.0'}
hasBin: true
dependencies:
- '@types/estree': 1.0.5
+ '@types/estree': 1.0.6
optionalDependencies:
- '@rollup/rollup-android-arm-eabi': 4.18.1
- '@rollup/rollup-android-arm64': 4.18.1
- '@rollup/rollup-darwin-arm64': 4.18.1
- '@rollup/rollup-darwin-x64': 4.18.1
- '@rollup/rollup-linux-arm-gnueabihf': 4.18.1
- '@rollup/rollup-linux-arm-musleabihf': 4.18.1
- '@rollup/rollup-linux-arm64-gnu': 4.18.1
- '@rollup/rollup-linux-arm64-musl': 4.18.1
- '@rollup/rollup-linux-powerpc64le-gnu': 4.18.1
- '@rollup/rollup-linux-riscv64-gnu': 4.18.1
- '@rollup/rollup-linux-s390x-gnu': 4.18.1
- '@rollup/rollup-linux-x64-gnu': 4.18.1
- '@rollup/rollup-linux-x64-musl': 4.18.1
- '@rollup/rollup-win32-arm64-msvc': 4.18.1
- '@rollup/rollup-win32-ia32-msvc': 4.18.1
- '@rollup/rollup-win32-x64-msvc': 4.18.1
+ '@rollup/rollup-android-arm-eabi': 4.24.0
+ '@rollup/rollup-android-arm64': 4.24.0
+ '@rollup/rollup-darwin-arm64': 4.24.0
+ '@rollup/rollup-darwin-x64': 4.24.0
+ '@rollup/rollup-linux-arm-gnueabihf': 4.24.0
+ '@rollup/rollup-linux-arm-musleabihf': 4.24.0
+ '@rollup/rollup-linux-arm64-gnu': 4.24.0
+ '@rollup/rollup-linux-arm64-musl': 4.24.0
+ '@rollup/rollup-linux-powerpc64le-gnu': 4.24.0
+ '@rollup/rollup-linux-riscv64-gnu': 4.24.0
+ '@rollup/rollup-linux-s390x-gnu': 4.24.0
+ '@rollup/rollup-linux-x64-gnu': 4.24.0
+ '@rollup/rollup-linux-x64-musl': 4.24.0
+ '@rollup/rollup-win32-arm64-msvc': 4.24.0
+ '@rollup/rollup-win32-ia32-msvc': 4.24.0
+ '@rollup/rollup-win32-x64-msvc': 4.24.0
fsevents: 2.3.3
dev: true
@@ -5538,14 +5268,14 @@ packages:
hasBin: true
dev: true
- /semver@7.6.2:
- resolution: {integrity: sha512-FNAIBWCx9qcRhoHcgcJ0gvU7SN1lYU2ZXuSfl04bSC5OpvDHFyJCjdNHomPXxjQlCBU67YW64PzY7/VIEH7F2w==}
+ /semver@7.6.3:
+ resolution: {integrity: sha512-oVekP1cKtI+CTDvHWYFUcMtsK/00wmAEfyqKfNdARm8u1wNVhSgaX7A8d4UuIlUI5e84iEwOhs7ZPYRmzU9U6A==}
engines: {node: '>=10'}
hasBin: true
dev: true
- /send@0.18.0:
- resolution: {integrity: sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg==}
+ /send@0.19.0:
+ resolution: {integrity: sha512-dW41u5VfLXu8SJh5bwRmyYUbAoSB3c9uQh6L8h/KtsFREPWpbX1lrljJo186Jc4nmci/sGUZ9a0a0J2zgfq2hw==}
engines: {node: '>= 0.8.0'}
dependencies:
debug: 2.6.9
@@ -5565,14 +5295,14 @@ packages:
- supports-color
dev: true
- /serve-static@1.15.0:
- resolution: {integrity: sha512-XGuRDNjXUijsUL0vl6nSD7cwURuzEgglbOaFuZM9g3kwDXOWVTck0jLzjPzGD+TazWbboZYu52/9/XPdUgne9g==}
+ /serve-static@1.16.2:
+ resolution: {integrity: sha512-VqpjJZKadQB/PEbEwvFdO43Ax5dFBZ2UECszz8bQ7pi7wt//PWe1P6MN7eCnjsatYtBT6EuiClbjSWP2WrIoTw==}
engines: {node: '>= 0.8.0'}
dependencies:
- encodeurl: 1.0.2
+ encodeurl: 2.0.0
escape-html: 1.0.3
parseurl: 1.3.3
- send: 0.18.0
+ send: 0.19.0
transitivePeerDependencies:
- supports-color
dev: true
@@ -5680,15 +5410,8 @@ packages:
is-fullwidth-code-point: 3.0.0
dev: true
- /snake-case@3.0.4:
- resolution: {integrity: sha512-LAOh4z89bGQvl9pFfNF8V146i7o7/CqFPbqzYgP+yYzDIDeS9HaNFtXABamRW+AQzEVODcvE79ljJ+8a9YSdMg==}
- dependencies:
- dot-case: 3.0.4
- tslib: 2.6.3
- dev: true
-
- /source-map-js@1.2.0:
- resolution: {integrity: sha512-itJW8lvSA0TXEphiRoawsCksnlf8SyvmFzIhltqAHluXd88pkCd+cXJVHTDwdCr0IzwptSm035IHQktUu1QUMg==}
+ /source-map-js@1.2.1:
+ resolution: {integrity: sha512-UXWMKhLOwVKb728IUtQPXxfYU+usdybtUrK/8uGE8CQMvrhOpwvzDBwj0QhSL7MQc7vIsISBG8VQ8+IDQxpfQA==}
engines: {node: '>=0.10.0'}
dev: true
@@ -5707,7 +5430,7 @@ packages:
resolution: {integrity: sha512-kN9dJbvnySHULIluDHy32WHRUu3Og7B9sbY7tsFLctQkIqnMh3hErYgdMjTYuqmcXX+lK5T1lnUt3G7zNswmZA==}
dependencies:
spdx-expression-parse: 3.0.1
- spdx-license-ids: 3.0.18
+ spdx-license-ids: 3.0.20
dev: true
/spdx-exceptions@2.5.0:
@@ -5718,11 +5441,11 @@ packages:
resolution: {integrity: sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q==}
dependencies:
spdx-exceptions: 2.5.0
- spdx-license-ids: 3.0.18
+ spdx-license-ids: 3.0.20
dev: true
- /spdx-license-ids@3.0.18:
- resolution: {integrity: sha512-xxRs31BqRYHwiMzudOrpSiHtZ8i/GeionCBDSilhYRj+9gIcI8wCZTlXZKu9vZIVqViP3dcp9qE5G6AlIaD+TQ==}
+ /spdx-license-ids@3.0.20:
+ resolution: {integrity: sha512-jg25NiDV/1fLtSgEgyvVyDunvaNHbuwF9lfNV17gSmPFAlYzdfNBlLtLzXTevwkPj7DhGbmN9VnmJIgLnhvaBw==}
dev: true
/sprintf-js@1.0.3:
@@ -5865,7 +5588,7 @@ packages:
resolution: {integrity: sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==}
engines: {node: '>=12'}
dependencies:
- ansi-regex: 6.0.1
+ ansi-regex: 6.1.0
dev: true
/strip-bom@3.0.0:
@@ -5918,6 +5641,7 @@ packages:
engines: {node: '>=4'}
dependencies:
has-flag: 3.0.0
+ dev: true
/supports-color@7.2.0:
resolution: {integrity: sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==}
@@ -5930,15 +5654,11 @@ packages:
resolution: {integrity: sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w==}
engines: {node: '>= 0.4'}
- /svg-parser@2.0.4:
- resolution: {integrity: sha512-e4hG1hRwoOdRb37cIMSgzNsxyzKfayW6VOflrwvR+/bzrkyxY/31WkbgnQpgtrNp1SdpJvpUAGTa/ZoiPNDuRQ==}
- dev: true
-
/table@6.8.2:
resolution: {integrity: sha512-w2sfv80nrAh2VCbqR5AK27wswXhqcck2AhfnNW76beQXskGZ1V12GwS//yYVa3d3fcvAip2OUnbDAjW2k3v9fA==}
engines: {node: '>=10.0.0'}
dependencies:
- ajv: 8.16.0
+ ajv: 8.17.1
lodash.truncate: 4.4.2
slice-ansi: 4.0.0
string-width: 4.2.3
@@ -5977,8 +5697,8 @@ packages:
resolution: {integrity: sha512-lBN9zLN/oAf68o3zNXYrdCt1kP8WsiGW8Oo2ka41b2IM5JL/S1CTyX1rW0mb/zSuJun0ZUrDxx4sqvYS2FWzPA==}
dev: false
- /tinybench@2.8.0:
- resolution: {integrity: sha512-1/eK7zUnIklz4JUUlL+658n58XO2hHLQfSk1Zf2LKieUjxidN16eKFEoDEfjHc3ohofSSqK3X5yO6VGb6iW8Lw==}
+ /tinybench@2.9.0:
+ resolution: {integrity: sha512-0+DUvqWMValLmha6lr4kD8iAMK1HzV0/aKnCtWb9v9641TnP/MFb7Pc2bxoxQjTXAErryXVgUOfv2YqNllqGeg==}
dev: true
/tinypool@0.8.4:
@@ -5991,10 +5711,6 @@ packages:
engines: {node: '>=14.0.0'}
dev: true
- /to-fast-properties@2.0.0:
- resolution: {integrity: sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog==}
- engines: {node: '>=4'}
-
/to-readable-stream@1.0.0:
resolution: {integrity: sha512-Iq25XBt6zD5npPhlLVXGFN3/gyR2/qODcKNNyTMd4vbm39HUaOiAM4PMq0eMVC/Tkxz+Zjdsc55g9yyz+Yq00Q==}
engines: {node: '>=6'}
@@ -6044,10 +5760,6 @@ packages:
resolution: {integrity: sha512-N82ooyxVNm6h1riLCoyS9e3fuJ3AMG2zIZs2Gd1ATcSFjSA23Q0fzjjZeh0jbJvWVDZ0cJT8yaNNaaXHzueNjg==}
dev: false
- /tslib@2.6.3:
- resolution: {integrity: sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==}
- dev: true
-
/tsutils@3.21.0(typescript@4.9.5):
resolution: {integrity: sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA==}
engines: {node: '>= 6'}
@@ -6075,8 +5787,8 @@ packages:
prelude-ls: 1.2.1
dev: true
- /type-detect@4.0.8:
- resolution: {integrity: sha512-0fr/mIH1dlO+x7TlcMy+bIDqKPsw/70tVyeHW787goQjhmqaZe10uwLujubK9q9Lg6Fiho1KUKDYz0Z7k7g5/g==}
+ /type-detect@4.1.0:
+ resolution: {integrity: sha512-Acylog8/luQ8L7il+geoSxhEkazvkslg7PSNKOX59mbB9cOveP5aq9h74Y7YU8yDpJwetzQQrfIwtf4Wp4LKcw==}
engines: {node: '>=4'}
dev: true
@@ -6147,8 +5859,8 @@ packages:
engines: {node: '>=4.2.0'}
hasBin: true
- /ufo@1.5.3:
- resolution: {integrity: sha512-Y7HYmWaFwPUmkoQCUIAYpKqkOf+SbVj/2fJJZ4RJMCfZp0rTGwRbzQD+HghfnhKOjL9E01okqz+ncJskGYfBNw==}
+ /ufo@1.5.4:
+ resolution: {integrity: sha512-UsUk3byDzKd04EyoZ7U4DOlxQaD14JUKQl6/P7wiX4FNvUfm3XL246n9W5AmqwW5RSFJ27NAuM0iLscAOYUiGQ==}
dev: true
/unbox-primitive@1.0.2:
@@ -6160,8 +5872,8 @@ packages:
which-boxed-primitive: 1.0.2
dev: true
- /undici-types@5.26.5:
- resolution: {integrity: sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==}
+ /undici-types@6.19.8:
+ resolution: {integrity: sha512-ve2KP6f/JnbPBFyobGHuerC9g1FYGn/F8n1LWTwNxCEzd6IfqTwUQcNXgEtmmQ6DlRrC1hrSrBnCZPokRrDHjw==}
dev: true
/unique-string@1.0.0:
@@ -6176,17 +5888,6 @@ packages:
engines: {node: '>= 0.8'}
dev: true
- /update-browserslist-db@1.1.0(browserslist@4.23.2):
- resolution: {integrity: sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ==}
- hasBin: true
- peerDependencies:
- browserslist: '>= 4.21.0'
- dependencies:
- browserslist: 4.23.2
- escalade: 3.1.2
- picocolors: 1.0.1
- dev: true
-
/update-notifier@3.0.1:
resolution: {integrity: sha512-grrmrB6Zb8DUiyDIaeRTBCkgISYUgETNe7NglEbVsrLWXeESnlCSP50WfRSj/GmzMPl6Uchj24S/p80nP/ZQrQ==}
engines: {node: '>=8'}
@@ -6264,42 +5965,28 @@ packages:
hasBin: true
dependencies:
cac: 6.7.14
- debug: 4.3.5
+ debug: 4.3.7
pathe: 1.1.2
- picocolors: 1.0.1
- vite: 5.3.3(less@3.13.1)
+ picocolors: 1.1.1
+ vite: 5.4.10(less@3.13.1)
transitivePeerDependencies:
- '@types/node'
- less
- lightningcss
- sass
+ - sass-embedded
- stylus
- sugarss
- supports-color
- terser
dev: true
- /vite-plugin-svgr@4.2.0(typescript@4.9.5)(vite@4.5.3):
- resolution: {integrity: sha512-SC7+FfVtNQk7So0XMjrrtLAbEC8qjFPifyD7+fs/E6aaNdVde6umlVVh0QuwDLdOMu7vp5RiGFsB70nj5yo0XA==}
- peerDependencies:
- vite: ^2.6.0 || 3 || 4 || 5
- dependencies:
- '@rollup/pluginutils': 5.1.0
- '@svgr/core': 8.1.0(typescript@4.9.5)
- '@svgr/plugin-jsx': 8.1.0(@svgr/core@8.1.0)
- vite: 4.5.3(less@3.13.1)
- transitivePeerDependencies:
- - rollup
- - supports-color
- - typescript
- dev: true
-
/vite-tsconfig-paths@3.6.0(vite@4.5.3):
resolution: {integrity: sha512-UfsPYonxLqPD633X8cWcPFVuYzx/CMNHAjZTasYwX69sXpa4gNmQkR0XCjj82h7zhLGdTWagMjC1qfb9S+zv0A==}
peerDependencies:
vite: '>2.0.0-0'
dependencies:
- debug: 4.3.5
+ debug: 4.3.7
globrex: 0.1.2
recrawl-sync: 2.2.3
tsconfig-paths: 4.2.0
@@ -6338,14 +6025,14 @@ packages:
dependencies:
esbuild: 0.18.20
less: 3.13.1
- postcss: 8.4.39
- rollup: 3.29.4
+ postcss: 8.4.47
+ rollup: 3.29.5
optionalDependencies:
fsevents: 2.3.3
dev: true
- /vite@5.3.3(less@3.13.1):
- resolution: {integrity: sha512-NPQdeCU0Dv2z5fu+ULotpuq5yfCS1BzKUIPhNbP3YBfAMGJXbt2nS+sbTFu+qchaqWTD+H3JK++nRwr6XIcp6A==}
+ /vite@5.4.10(less@3.13.1):
+ resolution: {integrity: sha512-1hvaPshuPUtxeQ0hsVH3Mud0ZanOLwVTneA1EgbAM5LhaZEqyPWGRQ7BtaMvUrTDeEaC8pxtj6a6jku3x4z6SQ==}
engines: {node: ^18.0.0 || >=20.0.0}
hasBin: true
peerDependencies:
@@ -6353,6 +6040,7 @@ packages:
less: '*'
lightningcss: ^1.21.0
sass: '*'
+ sass-embedded: '*'
stylus: '*'
sugarss: '*'
terser: ^5.4.0
@@ -6365,6 +6053,8 @@ packages:
optional: true
sass:
optional: true
+ sass-embedded:
+ optional: true
stylus:
optional: true
sugarss:
@@ -6374,8 +6064,8 @@ packages:
dependencies:
esbuild: 0.21.5
less: 3.13.1
- postcss: 8.4.39
- rollup: 4.18.1
+ postcss: 8.4.47
+ rollup: 4.24.0
optionalDependencies:
fsevents: 2.3.3
dev: true
@@ -6410,25 +6100,26 @@ packages:
'@vitest/snapshot': 1.6.0
'@vitest/spy': 1.6.0
'@vitest/utils': 1.6.0
- acorn-walk: 8.3.3
- chai: 4.4.1
- debug: 4.3.5
+ acorn-walk: 8.3.4
+ chai: 4.5.0
+ debug: 4.3.7
execa: 8.0.1
local-pkg: 0.5.0
- magic-string: 0.30.10
+ magic-string: 0.30.12
pathe: 1.1.2
- picocolors: 1.0.1
+ picocolors: 1.1.1
std-env: 3.7.0
strip-literal: 2.1.0
- tinybench: 2.8.0
+ tinybench: 2.9.0
tinypool: 0.8.4
- vite: 5.3.3(less@3.13.1)
+ vite: 5.4.10(less@3.13.1)
vite-node: 1.6.0(less@3.13.1)
why-is-node-running: 2.3.0
transitivePeerDependencies:
- less
- lightningcss
- sass
+ - sass-embedded
- stylus
- sugarss
- supports-color
@@ -6554,10 +6245,6 @@ packages:
resolution: {integrity: sha512-ncTzHV7NvsQZkYe1DW7cbDLm0YpzHmZF5r/iyP3ZnQtMiJ+pjzisCiMNI+Sj+xQF5pXhSHxSB3uDbsBTzY/c2A==}
dev: true
- /yallist@3.1.1:
- resolution: {integrity: sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==}
- dev: true
-
/yaml@1.10.2:
resolution: {integrity: sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg==}
engines: {node: '>= 6'}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts
index 37312098eccd..ad9e1fba4185 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts
@@ -17,7 +17,7 @@
import { defineConfig, splitVendorChunkPlugin } from 'vite';
import { resolve } from 'path';
-import react from '@vitejs/plugin-react';
+import react from '@vitejs/plugin-react-swc';
function pathResolve(dir: string) {
return resolve(__dirname, '.', dir)
@@ -25,6 +25,12 @@ function pathResolve(dir: string) {
// https://vitejs.dev/config/
export default defineConfig({
+ plugins: [
+ react({
+ devTarget: "es2015" //SWC by default bypasses the build target, set dev target explicitly
+ }),
+ splitVendorChunkPlugin()
+ ],
build: {
target: "es2015",
outDir: 'build',
@@ -44,7 +50,6 @@ export default defineConfig({
}
}
},
- plugins: [react(), splitVendorChunkPlugin()],
server: {
proxy: {
"/api": {