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": {