diff --git a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java index f966409c89c..400d36d9fa8 100644 --- a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java +++ b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java @@ -44,6 +44,7 @@ import org.apache.celeborn.client.read.MetricsCallback; import org.apache.celeborn.common.CelebornConf; import org.apache.celeborn.common.exception.CelebornIOException; +import org.apache.celeborn.common.exception.CelebornRuntimeException; import org.apache.celeborn.common.identity.UserIdentifier; import org.apache.celeborn.common.metrics.source.Role; import org.apache.celeborn.common.network.TransportContext; @@ -84,6 +85,8 @@ public class ShuffleClientImpl extends ShuffleClient { private final int registerShuffleMaxRetries; private final long registerShuffleRetryWaitMs; + private final int rpcMaxRetries; + private final long rpcRetryWait; private final int maxReviveTimes; private final boolean testRetryRevive; private final int pushBufferMaxSize; @@ -189,6 +192,8 @@ public ShuffleClientImpl(String appUniqueId, CelebornConf conf, UserIdentifier u this.userIdentifier = userIdentifier; registerShuffleMaxRetries = conf.clientRegisterShuffleMaxRetry(); registerShuffleRetryWaitMs = conf.clientRegisterShuffleRetryWaitMs(); + rpcMaxRetries = conf.clientRpcMaxRetries(); + rpcRetryWait = conf.clientRpcRetryWait(); maxReviveTimes = conf.clientPushMaxReviveTimes(); testRetryRevive = conf.testRetryRevive(); pushBufferMaxSize = conf.clientPushBufferMaxSize(); @@ -546,6 +551,8 @@ private ConcurrentHashMap registerShuffle( lifecycleManagerRef.askSync( RegisterShuffle$.MODULE$.apply(shuffleId, numMappers, numPartitions), conf.clientRpcRegisterShuffleAskTimeout(), + rpcMaxRetries, + rpcRetryWait, ClassTag$.MODULE$.apply(PbRegisterShuffleResponse.class))); } @@ -1747,7 +1754,9 @@ private void mapEndInternal( numMappers, partitionId, pushState.getFailedBatches()), - ClassTag$.MODULE$.apply(MapperEndResponse.class)); + rpcMaxRetries, + rpcRetryWait, + ClassTag$.MODULE$.apply(MapperEndResponse.class)); if (response.status() != StatusCode.SUCCESS) { throw new CelebornIOException("MapperEnd failed! StatusCode: " + response.status()); } @@ -1781,75 +1790,64 @@ public boolean cleanupShuffle(int shuffleId) { protected Tuple3 loadFileGroupInternal( int shuffleId, boolean isSegmentGranularityVisible) { - { - long getReducerFileGroupStartTime = System.nanoTime(); - String exceptionMsg = null; - Exception exception = null; - try { - if (lifecycleManagerRef == null) { - exceptionMsg = "Driver endpoint is null!"; + long getReducerFileGroupStartTime = System.nanoTime(); + String exceptionMsg = null; + Exception exception = null; + if (lifecycleManagerRef == null) { + exceptionMsg = "Driver endpoint is null!"; + logger.warn(exceptionMsg); + return Tuple3.apply(null, exceptionMsg, exception); + } + try { + GetReducerFileGroup getReducerFileGroup = + new GetReducerFileGroup(shuffleId, isSegmentGranularityVisible); + GetReducerFileGroupResponse response = + lifecycleManagerRef.askSync( + getReducerFileGroup, + conf.clientRpcGetReducerFileGroupAskTimeout(), + rpcMaxRetries, + rpcRetryWait, + ClassTag$.MODULE$.apply(GetReducerFileGroupResponse.class)); + switch (response.status()) { + case SUCCESS: + logger.info( + "Shuffle {} request reducer file group success using {} ms, result partition size {}.", + shuffleId, + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - getReducerFileGroupStartTime), + response.fileGroup().size()); + return Tuple3.apply( + new ReduceFileGroups( + response.fileGroup(), response.attempts(), response.partitionIds(),response.pushFailedBatches()), + null, + null); + case SHUFFLE_NOT_REGISTERED: + logger.warn( + "Request {} return {} for {}.", getReducerFileGroup, response.status(), shuffleId); + // return empty result + return Tuple3.apply( + new ReduceFileGroups( + response.fileGroup(), response.attempts(), response.partitionIds(),response.pushFailedBatches()), + null, + null); + case STAGE_END_TIME_OUT: + case SHUFFLE_DATA_LOST: + exceptionMsg = + String.format( + "Request %s return %s for %s.", + getReducerFileGroup, response.status(), shuffleId); logger.warn(exceptionMsg); - } else { - GetReducerFileGroup getReducerFileGroup = - new GetReducerFileGroup(shuffleId, isSegmentGranularityVisible); - - GetReducerFileGroupResponse response = - lifecycleManagerRef.askSync( - getReducerFileGroup, - conf.clientRpcGetReducerFileGroupAskTimeout(), - ClassTag$.MODULE$.apply(GetReducerFileGroupResponse.class)); - - switch (response.status()) { - case SUCCESS: - logger.info( - "Shuffle {} request reducer file group success using {} ms, result partition size {}.", - shuffleId, - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - getReducerFileGroupStartTime), - response.fileGroup().size()); - return Tuple3.apply( - new ReduceFileGroups( - response.fileGroup(), - response.attempts(), - response.partitionIds(), - response.pushFailedBatches()), - null, - null); - case SHUFFLE_NOT_REGISTERED: - logger.warn( - "Request {} return {} for {}.", - getReducerFileGroup, - response.status(), - shuffleId); - // return empty result - return Tuple3.apply( - new ReduceFileGroups( - response.fileGroup(), - response.attempts(), - response.partitionIds(), - response.pushFailedBatches()), - null, - null); - case STAGE_END_TIME_OUT: - case SHUFFLE_DATA_LOST: - exceptionMsg = - String.format( - "Request %s return %s for %s.", - getReducerFileGroup, response.status(), shuffleId); - logger.warn(exceptionMsg); - break; - default: // fall out - } - } - } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - logger.error("Exception raised while call GetReducerFileGroup for {}.", shuffleId, e); - exceptionMsg = e.getMessage(); - exception = e; + break; + default: // fall out } - return Tuple3.apply(null, exceptionMsg, exception); + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + logger.error("Exception raised while call GetReducerFileGroup for {}.", shuffleId, e); + exceptionMsg = e.getMessage(); + exception = e; } + return Tuple3.apply(null, exceptionMsg, exception); } @Override @@ -1985,8 +1983,17 @@ public void shutdown() { @Override public void setupLifecycleManagerRef(String host, int port) { logger.info("setupLifecycleManagerRef: host = {}, port = {}", host, port); - lifecycleManagerRef = - rpcEnv.setupEndpointRef(new RpcAddress(host, port), RpcNameConstants.LIFECYCLE_MANAGER_EP); + try { + lifecycleManagerRef = + rpcEnv.setupEndpointRef( + new RpcAddress(host, port), + RpcNameConstants.LIFECYCLE_MANAGER_EP, + rpcMaxRetries, + rpcRetryWait); + } catch (Exception e) { + throw new CelebornRuntimeException("setupLifecycleManagerRef failed!", e); + } + initDataClientFactoryIfNeeded(); } diff --git a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java index 3d2934df710..ea3df88d95a 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java +++ b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java @@ -556,7 +556,14 @@ private PartitionReader createReader( logger.debug("Read local shuffle file {}", localHostAddress); containLocalRead = true; return new LocalPartitionReader( - conf, shuffleKey, location, clientFactory, startMapIndex, endMapIndex, callback); + conf, + shuffleKey, + location, + pbStreamHandler, + clientFactory, + startMapIndex, + endMapIndex, + callback); } else { return new WorkerPartitionReader( conf, @@ -575,7 +582,14 @@ private PartitionReader createReader( case S3: case HDFS: return new DfsPartitionReader( - conf, shuffleKey, location, clientFactory, startMapIndex, endMapIndex, callback); + conf, + shuffleKey, + location, + pbStreamHandler, + clientFactory, + startMapIndex, + endMapIndex, + callback); default: throw new CelebornIOException( String.format("Unknown storage info %s to read location %s", storageInfo, location)); diff --git a/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java b/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java index b69cf580fa7..313fe77d1c4 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java +++ b/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java @@ -76,6 +76,7 @@ public DfsPartitionReader( CelebornConf conf, String shuffleKey, PartitionLocation location, + PbStreamHandler pbStreamHandler, TransportClientFactory clientFactory, int startMapIndex, int endMapIndex, @@ -95,10 +96,10 @@ public DfsPartitionReader( this.hadoopFs = ShuffleClient.getHadoopFs(conf).get(StorageInfo.Type.HDFS); } - if (endMapIndex != Integer.MAX_VALUE) { - long fetchTimeoutMs = conf.clientFetchTimeoutMs(); - try { - client = clientFactory.createClient(location.getHost(), location.getFetchPort()); + long fetchTimeoutMs = conf.clientFetchTimeoutMs(); + try { + client = clientFactory.createClient(location.getHost(), location.getFetchPort()); + if (pbStreamHandler == null) { TransportMessage openStream = new TransportMessage( MessageType.OPEN_STREAM, @@ -112,13 +113,16 @@ public DfsPartitionReader( ByteBuffer response = client.sendRpcSync(openStream.toByteBuffer(), fetchTimeoutMs); streamHandler = TransportMessage.fromByteBuffer(response).getParsedPayload(); // Parse this message to ensure sort is done. - } catch (IOException | InterruptedException e) { - throw new IOException( - "read shuffle file from DFS failed, filePath: " - + location.getStorageInfo().getFilePath(), - e); + } else { + streamHandler = pbStreamHandler; } + } catch (IOException | InterruptedException e) { + throw new IOException( + "read shuffle file from DFS failed, filePath: " + location.getStorageInfo().getFilePath(), + e); + } + if (endMapIndex != Integer.MAX_VALUE) { dfsInputStream = hadoopFs.open(new Path(Utils.getSortedFilePath(location.getStorageInfo().getFilePath()))); chunkOffsets.addAll( diff --git a/client/src/main/java/org/apache/celeborn/client/read/LocalPartitionReader.java b/client/src/main/java/org/apache/celeborn/client/read/LocalPartitionReader.java index a769687c8df..722bab100cf 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/LocalPartitionReader.java +++ b/client/src/main/java/org/apache/celeborn/client/read/LocalPartitionReader.java @@ -74,6 +74,7 @@ public LocalPartitionReader( CelebornConf conf, String shuffleKey, PartitionLocation location, + PbStreamHandler pbStreamHandler, TransportClientFactory clientFactory, int startMapIndex, int endMapIndex, @@ -95,19 +96,23 @@ public LocalPartitionReader( long fetchTimeoutMs = conf.clientFetchTimeoutMs(); try { client = clientFactory.createClient(location.getHost(), location.getFetchPort(), 0); - TransportMessage openStreamMsg = - new TransportMessage( - MessageType.OPEN_STREAM, - PbOpenStream.newBuilder() - .setShuffleKey(shuffleKey) - .setFileName(location.getFileName()) - .setStartIndex(startMapIndex) - .setEndIndex(endMapIndex) - .setReadLocalShuffle(true) - .build() - .toByteArray()); - ByteBuffer response = client.sendRpcSync(openStreamMsg.toByteBuffer(), fetchTimeoutMs); - streamHandler = TransportMessage.fromByteBuffer(response).getParsedPayload(); + if (pbStreamHandler == null) { + TransportMessage openStreamMsg = + new TransportMessage( + MessageType.OPEN_STREAM, + PbOpenStream.newBuilder() + .setShuffleKey(shuffleKey) + .setFileName(location.getFileName()) + .setStartIndex(startMapIndex) + .setEndIndex(endMapIndex) + .setReadLocalShuffle(true) + .build() + .toByteArray()); + ByteBuffer response = client.sendRpcSync(openStreamMsg.toByteBuffer(), fetchTimeoutMs); + streamHandler = TransportMessage.fromByteBuffer(response).getParsedPayload(); + } else { + this.streamHandler = pbStreamHandler; + } } catch (IOException | InterruptedException e) { throw new IOException( "Read shuffle file from local file failed, partition location: " diff --git a/client/src/test/java/org/apache/celeborn/client/ShuffleClientSuiteJ.java b/client/src/test/java/org/apache/celeborn/client/ShuffleClientSuiteJ.java index 477a3c19235..7053937a06a 100644 --- a/client/src/test/java/org/apache/celeborn/client/ShuffleClientSuiteJ.java +++ b/client/src/test/java/org/apache/celeborn/client/ShuffleClientSuiteJ.java @@ -258,6 +258,12 @@ private CelebornConf setupEnv( RegisterShuffleResponse$.MODULE$.apply( statusCode, new PartitionLocation[] {primaryLocation})); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + t -> + RegisterShuffleResponse$.MODULE$.apply( + statusCode, new PartitionLocation[] {primaryLocation})); + shuffleClient.setupLifecycleManagerRef(endpointRef); ChannelFuture mockedFuture = @@ -424,6 +430,14 @@ public void testUpdateReducerFileGroupInterrupted() throws InterruptedException Collections.emptyMap()); }); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + t -> { + Thread.sleep(60 * 1000); + return GetReducerFileGroupResponse$.MODULE$.apply( + StatusCode.SUCCESS, locations, new int[0], Collections.emptySet()); + }); + shuffleClient = new ShuffleClientImpl(TEST_APPLICATION_ID, conf, new UserIdentifier("mock", "mock")); shuffleClient.setupLifecycleManagerRef(endpointRef); @@ -467,6 +481,13 @@ public void testUpdateReducerFileGroupNonFetchFailureExceptions() { Collections.emptyMap()); }); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + t -> { + return GetReducerFileGroupResponse$.MODULE$.apply( + StatusCode.SHUFFLE_NOT_REGISTERED, locations, new int[0], Collections.emptySet()); + }); + shuffleClient = new ShuffleClientImpl(TEST_APPLICATION_ID, conf, new UserIdentifier("mock", "mock")); shuffleClient.setupLifecycleManagerRef(endpointRef); @@ -488,6 +509,13 @@ public void testUpdateReducerFileGroupNonFetchFailureExceptions() { Collections.emptyMap()); }); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + t -> { + return GetReducerFileGroupResponse$.MODULE$.apply( + StatusCode.STAGE_END_TIME_OUT, locations, new int[0], Collections.emptySet()); + }); + shuffleClient = new ShuffleClientImpl(TEST_APPLICATION_ID, conf, new UserIdentifier("mock", "mock")); shuffleClient.setupLifecycleManagerRef(endpointRef); @@ -509,6 +537,13 @@ public void testUpdateReducerFileGroupNonFetchFailureExceptions() { Collections.emptyMap()); }); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + t -> { + return GetReducerFileGroupResponse$.MODULE$.apply( + StatusCode.SHUFFLE_DATA_LOST, locations, new int[0], Collections.emptySet()); + }); + shuffleClient = new ShuffleClientImpl(TEST_APPLICATION_ID, conf, new UserIdentifier("mock", "mock")); shuffleClient.setupLifecycleManagerRef(endpointRef); @@ -531,6 +566,12 @@ public void testUpdateReducerFileGroupTimeout() throws InterruptedException { throw new RpcTimeoutException( "Rpc timeout", new TimeoutException("ask sync timeout")); }); + when(endpointRef.askSync(any(), any(), any(Integer.class), any(Long.class), any())) + .thenAnswer( + invocation -> { + throw new RpcTimeoutException( + "Rpc timeout", new TimeoutException("ask sync timeout")); + }); shuffleClient = new ShuffleClientImpl(TEST_APPLICATION_ID, conf, new UserIdentifier("mock", "mock")); diff --git a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala index eb750daeb87..0eae8a3f762 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -520,6 +520,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se new RpcTimeout(get(RPC_LOOKUP_TIMEOUT).milli, RPC_LOOKUP_TIMEOUT.key) def rpcAskTimeout: RpcTimeout = new RpcTimeout(get(RPC_ASK_TIMEOUT).milli, RPC_ASK_TIMEOUT.key) + def rpcRetryWait: Long = get(RPC_RETRY_WAIT) def rpcInMemoryBoundedInboxCapacity(): Int = { get(RPC_INBOX_CAPACITY) } @@ -1017,6 +1018,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def clientRpcCacheExpireTime: Long = get(CLIENT_RPC_CACHE_EXPIRE_TIME) def clientRpcSharedThreads: Int = get(CLIENT_RPC_SHARED_THREADS) def clientRpcMaxRetries: Int = get(CLIENT_RPC_MAX_RETIRES) + def clientRpcRetryWait: Long = get(CLIENT_RPC_RETRY_WAIT) def pushDataTimeoutMs: Long = get(CLIENT_PUSH_DATA_TIMEOUT) def clientPushLimitStrategy: String = get(CLIENT_PUSH_LIMIT_STRATEGY) def clientPushSlowStartInitialSleepTime: Long = get(CLIENT_PUSH_SLOW_START_INITIAL_SLEEP_TIME) @@ -1889,6 +1891,14 @@ object CelebornConf extends Logging { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("60s") + val RPC_RETRY_WAIT: ConfigEntry[Long] = + buildConf("celeborn.rpc.retryWait") + .categories("network") + .version("0.6.0") + .doc("Time to wait before next retry on RpcTimeoutException.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") + val RPC_DISPATCHER_THREADS: ConfigEntry[Int] = buildConf("celeborn.rpc.dispatcher.threads") .withAlternative("celeborn.rpc.dispatcher.numThreads") @@ -4940,6 +4950,14 @@ object CelebornConf extends Logging { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("3s") + val CLIENT_RPC_RETRY_WAIT: ConfigEntry[Long] = + buildConf("celeborn.client.rpc.retryWait") + .categories("client") + .version("0.6.0") + .doc("Client-specified time to wait before next retry on RpcTimeoutException.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") + val CLIENT_RESERVE_SLOTS_MAX_RETRIES: ConfigEntry[Int] = buildConf("celeborn.client.reserveSlots.maxRetries") .withAlternative("celeborn.slots.reserve.maxRetries") @@ -5089,7 +5107,7 @@ object CelebornConf extends Logging { buildConf("celeborn.client.rpc.maxRetries") .categories("client") .version("0.3.2") - .doc("Max RPC retry times in LifecycleManager.") + .doc("Max RPC retry times in client.") .intConf .createWithDefault(3) diff --git a/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEndpointRef.scala b/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEndpointRef.scala index edd7005e2e9..8c861cf57ab 100644 --- a/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEndpointRef.scala +++ b/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEndpointRef.scala @@ -17,6 +17,9 @@ package org.apache.celeborn.common.rpc +import java.util.Random +import java.util.concurrent.TimeUnit + import scala.concurrent.Future import scala.reflect.ClassTag @@ -30,6 +33,7 @@ abstract class RpcEndpointRef(conf: CelebornConf) extends Serializable with Logging { private[this] val defaultAskTimeout = conf.rpcAskTimeout + private[this] val defaultRetryWait = conf.rpcRetryWait /** * return the address for the [[RpcEndpointRef]] @@ -88,4 +92,67 @@ abstract class RpcEndpointRef(conf: CelebornConf) val future = ask[T](message, timeout) timeout.awaitResult(future, address) } + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * default timeout, retry if timeout, throw an exception if this still fails. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in a message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @param retryCount the number of retries for the timeout + * @param retryWait the waiting time for a retry + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askSync[T: ClassTag](message: Any, retryCount: Int, retryWait: Long = defaultRetryWait): T = + askSync(message, defaultAskTimeout, retryCount, retryWait) + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * specified timeout, retry if timeout, throw an exception if this still fails. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in a message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @param timeout the timeout duration + * @param retryCount the number of retries for the timeout + * @param retryWait the waiting time for a retry + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askSync[T: ClassTag]( + message: Any, + timeout: RpcTimeout, + retryCount: Int, + retryWait: Long): T = { + var numRetries = retryCount + while (numRetries > 0) { + numRetries -= 1 + try { + val future = ask[T](message, timeout) + return timeout.awaitResult(future, address) + } catch { + case e: RpcTimeoutException => + if (numRetries > 0) { + val random = new Random + val retryWaitMs = random.nextInt(retryWait.toInt) + try { + TimeUnit.MILLISECONDS.sleep(retryWaitMs) + } catch { + case _: InterruptedException => + throw e + } + } else { + throw e + } + case e: Exception => + throw e + } + } + // should never be here + null.asInstanceOf[T] + } } diff --git a/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEnv.scala b/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEnv.scala index 89973a936be..19f522a0afe 100644 --- a/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEnv.scala +++ b/common/src/main/scala/org/apache/celeborn/common/rpc/RpcEnv.scala @@ -18,6 +18,8 @@ package org.apache.celeborn.common.rpc import java.io.File +import java.util.Random +import java.util.concurrent.TimeUnit import scala.concurrent.Future @@ -109,6 +111,7 @@ object RpcEnv { abstract class RpcEnv(config: RpcEnvConfig) { private[celeborn] val defaultLookupTimeout = config.conf.rpcLookupTimeout + private[celeborn] val defaultRetryWait = config.conf.rpcRetryWait /** * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement @@ -147,6 +150,42 @@ abstract class RpcEnv(config: RpcEnvConfig) { setupEndpointRefByAddr(RpcEndpointAddress(address, endpointName)) } + /** + * Retrieve the [[RpcEndpointRef]] represented by `address` and `endpointName` with timeout retry. + * This is a blocking action. + */ + def setupEndpointRef( + address: RpcAddress, + endpointName: String, + retryCount: Int, + retryWait: Long = defaultRetryWait): RpcEndpointRef = { + var numRetries = retryCount + while (numRetries > 0) { + numRetries -= 1 + try { + return setupEndpointRefByAddr(RpcEndpointAddress(address, endpointName)) + } catch { + case e: RpcTimeoutException => + if (numRetries > 0) { + val random = new Random + val retryWaitMs = random.nextInt(retryWait.toInt) + try { + TimeUnit.MILLISECONDS.sleep(retryWaitMs) + } catch { + case _: InterruptedException => + throw e + } + } else { + throw e + } + case e: Exception => + throw e + } + } + // should never be here + null + } + /** * Stop [[RpcEndpoint]] specified by `endpoint`. */ diff --git a/cpp/celeborn/network/CMakeLists.txt b/cpp/celeborn/network/CMakeLists.txt index 1acf114bb64..d78a769cc16 100644 --- a/cpp/celeborn/network/CMakeLists.txt +++ b/cpp/celeborn/network/CMakeLists.txt @@ -16,13 +16,15 @@ add_library( network STATIC Message.cpp - MessageDispatcher.cpp) + MessageDispatcher.cpp + TransportClient.cpp) target_include_directories(network PUBLIC ${CMAKE_BINARY_DIR}) target_link_libraries( network memory + conf proto utils protocol diff --git a/cpp/celeborn/network/MessageDispatcher.h b/cpp/celeborn/network/MessageDispatcher.h index 24a233a7c69..676ef081ba2 100644 --- a/cpp/celeborn/network/MessageDispatcher.h +++ b/cpp/celeborn/network/MessageDispatcher.h @@ -52,62 +52,62 @@ class MessageDispatcher : public wangle::ClientDispatcherBase< SerializePipeline, std::unique_ptr, std::unique_ptr> { -public: - void read(Context*, std::unique_ptr toRecvMsg) override; + public: + void read(Context*, std::unique_ptr toRecvMsg) override; - virtual folly::Future> sendRpcRequest( - std::unique_ptr toSendMsg) { - return operator()(std::move(toSendMsg)); - } + virtual folly::Future> sendRpcRequest( + std::unique_ptr toSendMsg) { + return operator()(std::move(toSendMsg)); + } - virtual folly::Future> sendFetchChunkRequest( - const protocol::StreamChunkSlice& streamChunkSlice, - std::unique_ptr toSendMsg); + virtual folly::Future> sendFetchChunkRequest( + const protocol::StreamChunkSlice& streamChunkSlice, + std::unique_ptr toSendMsg); - virtual void sendRpcRequestWithoutResponse( - std::unique_ptr toSendMsg); + virtual void sendRpcRequestWithoutResponse( + std::unique_ptr toSendMsg); - folly::Future> operator()( - std::unique_ptr toSendMsg) override; + folly::Future> operator()( + std::unique_ptr toSendMsg) override; - void readEOF(Context* ctx) override; + void readEOF(Context* ctx) override; - void readException(Context* ctx, folly::exception_wrapper e) override; + void readException(Context* ctx, folly::exception_wrapper e) override; - void transportActive(Context* ctx) override; + void transportActive(Context* ctx) override; - void transportInactive(Context* ctx) override; + void transportInactive(Context* ctx) override; - folly::Future writeException( - Context* ctx, - folly::exception_wrapper e) override; + folly::Future writeException( + Context* ctx, + folly::exception_wrapper e) override; - folly::Future close() override; + folly::Future close() override; - folly::Future close(Context* ctx) override; + folly::Future close(Context* ctx) override; - bool isAvailable() override { - return !closed_; - } + bool isAvailable() override { + return !closed_; + } -private: - void cleanup(); + private: + void cleanup(); - using MsgPromise = folly::Promise>; - struct MsgPromiseHolder { - MsgPromise msgPromise; - std::chrono::time_point requestTime; - }; - folly::Synchronized, std::mutex> - requestIdRegistry_; - folly::Synchronized< - std::unordered_map< - protocol::StreamChunkSlice, - MsgPromiseHolder, - protocol::StreamChunkSlice::Hasher>, - std::mutex> - streamChunkSliceRegistry_; - std::atomic closed_{false}; + using MsgPromise = folly::Promise>; + struct MsgPromiseHolder { + MsgPromise msgPromise; + std::chrono::time_point requestTime; + }; + folly::Synchronized, std::mutex> + requestIdRegistry_; + folly::Synchronized< + std::unordered_map< + protocol::StreamChunkSlice, + MsgPromiseHolder, + protocol::StreamChunkSlice::Hasher>, + std::mutex> + streamChunkSliceRegistry_; + std::atomic closed_{false}; }; } // namespace network } // namespace celeborn diff --git a/cpp/celeborn/network/TransportClient.cpp b/cpp/celeborn/network/TransportClient.cpp new file mode 100644 index 00000000000..796eb5d5082 --- /dev/null +++ b/cpp/celeborn/network/TransportClient.cpp @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "celeborn/network/TransportClient.h" + +#include "celeborn/network/FrameDecoder.h" +#include "celeborn/protocol/TransportMessage.h" + +namespace celeborn { +namespace network { +void MessageSerializeHandler::read( + Context* ctx, + std::unique_ptr msg) { + auto buffer = memory::ByteBuffer::createReadOnly(std::move(msg)); + ctx->fireRead(Message::decodeFrom(std::move(buffer))); +} + +folly::Future MessageSerializeHandler::write( + Context* ctx, + std::unique_ptr msg) { + return ctx->fireWrite(msg->encode()->getData()); +} + +TransportClient::TransportClient( + std::unique_ptr> client, + std::unique_ptr dispatcher, + Timeout defaultTimeout) + : client_(std::move(client)), + dispatcher_(std::move(dispatcher)), + defaultTimeout_(defaultTimeout) {} + +RpcResponse TransportClient::sendRpcRequestSync( + const RpcRequest& request, + Timeout timeout) { + try { + auto requestMsg = std::make_unique(request); + auto future = dispatcher_->sendRpcRequest(std::move(requestMsg)); + auto responseMsg = std::move(future).get(timeout); + CELEBORN_CHECK( + responseMsg->type() == Message::RPC_RESPONSE, + "responseMsg type should be RPC_RESPONSE"); + return RpcResponse(*reinterpret_cast(responseMsg.get())); + } catch (const std::exception& e) { + std::string errorMsg = fmt::format( + "sendRpc failure, requestId: {}, timeout: {}, errorMsg: {}", + request.requestId(), + timeout, + folly::exceptionStr(e).toStdString()); + LOG(ERROR) << errorMsg; + CELEBORN_FAIL(errorMsg); + } +} + +void TransportClient::sendRpcRequestWithoutResponse(const RpcRequest& request) { + try { + auto requestMsg = std::make_unique(request); + dispatcher_->sendRpcRequestWithoutResponse(std::move(requestMsg)); + } catch (const std::exception& e) { + std::string errorMsg = fmt::format( + "sendRpc failure, requestId: {}, errorMsg: {}", + request.requestId(), + folly::exceptionStr(e).toStdString()); + LOG(ERROR) << errorMsg; + CELEBORN_FAIL(errorMsg); + } +} + +void TransportClient::fetchChunkAsync( + const protocol::StreamChunkSlice& streamChunkSlice, + const RpcRequest& request, + FetchChunkSuccessCallback onSuccess, + FetchChunkFailureCallback onFailure) { + try { + auto requestMsg = std::make_unique(request); + auto future = dispatcher_->sendFetchChunkRequest( + streamChunkSlice, std::move(requestMsg)); + std::move(future) + .thenValue([=, _onSuccess = onSuccess, _onFailure = onFailure]( + std::unique_ptr responseMsg) { + if (responseMsg->type() == Message::CHUNK_FETCH_SUCCESS) { + auto chunkFetchSuccess = + reinterpret_cast(responseMsg.get()); + _onSuccess(streamChunkSlice, chunkFetchSuccess->body()); + } else { + _onFailure( + streamChunkSlice, + std::make_unique(fmt::format( + "chunk fetch of streamChunkSlice {} does not succeed", + streamChunkSlice.toString()))); + } + }) + .thenError( + [=, _onFailure = onFailure](const folly::exception_wrapper& e) { + _onFailure( + streamChunkSlice, + std::make_unique(e.what().toStdString())); + }); + } catch (std::exception& e) { + CELEBORN_FAIL(e.what()); + } +} + +SerializePipeline::Ptr MessagePipelineFactory::newPipeline( + std::shared_ptr sock) { + auto pipeline = SerializePipeline::create(); + pipeline->addBack(wangle::AsyncSocketHandler(sock)); + // Ensure we can write from any thread. + pipeline->addBack(wangle::EventBaseHandler()); + pipeline->addBack(FrameDecoder()); + pipeline->addBack(MessageSerializeHandler()); + pipeline->finalize(); + + return pipeline; +} + +TransportClientFactory::TransportClientFactory( + const std::shared_ptr& conf) { + numConnectionsPerPeer_ = conf->networkIoNumConnectionsPerPeer(); + rpcLookupTimeout_ = conf->rpcLookupTimeout(); + connectTimeout_ = conf->networkConnectTimeout(); + numClientThreads_ = conf->networkIoClientThreads(); + if (numClientThreads_ <= 0) { + numClientThreads_ = std::thread::hardware_concurrency() * 2; + } + clientExecutor_ = + std::make_shared(numClientThreads_); +} + +std::shared_ptr TransportClientFactory::createClient( + const std::string& host, + uint16_t port) { + auto address = folly::SocketAddress(host, port); + auto pool = clientPools_.withLock([&](auto& registry) { + auto iter = registry.find(address); + if (iter != registry.end()) { + return iter->second; + } + auto createdPool = std::make_shared(); + createdPool->clients.resize(numConnectionsPerPeer_); + registry[address] = createdPool; + return createdPool; + }); + auto clientId = std::rand() % numConnectionsPerPeer_; + { + std::lock_guard lock(pool->mutex); + // TODO: auto-disconnect if the connection is idle for a long time? + if (pool->clients[clientId] && pool->clients[clientId]->active()) { + VLOG(1) << "reusing client for address host " << host << " port " << port; + return pool->clients[clientId]; + } + + auto bootstrap = + std::make_unique>(); + bootstrap->group(clientExecutor_); + bootstrap->pipelineFactory(std::make_shared()); + try { + auto pipeline = bootstrap->connect(folly::SocketAddress(host, port)) + .get(rpcLookupTimeout_); + + auto dispatcher = std::make_unique(); + dispatcher->setPipeline(pipeline); + pool->clients[clientId] = std::make_shared( + std::move(bootstrap), std::move(dispatcher), connectTimeout_); + return pool->clients[clientId]; + } catch (const std::exception& e) { + std::string errorMsg = fmt::format( + "connect to server failure, serverAddr: {}:{}, timeout: {}, errorMsg: {}", + host, + port, + connectTimeout_, + folly::exceptionStr(e).toStdString()); + LOG(ERROR) << errorMsg; + CELEBORN_FAIL(errorMsg); + } + } +} +} // namespace network +} // namespace celeborn diff --git a/cpp/celeborn/network/TransportClient.h b/cpp/celeborn/network/TransportClient.h new file mode 100644 index 00000000000..87e107e372f --- /dev/null +++ b/cpp/celeborn/network/TransportClient.h @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include +#include +#include + +#include "celeborn/conf/CelebornConf.h" +#include "celeborn/network/Message.h" +#include "celeborn/network/MessageDispatcher.h" +#include "celeborn/protocol/ControlMessages.h" +#include "celeborn/utils/CelebornUtils.h" + +namespace celeborn { +namespace network { +/** + * MessageSerializeHandler serializes Message into folly::IOBuf when write(), + * and deserializes folly::IOBuf into Message when read(). + */ +class MessageSerializeHandler : public wangle::Handler< + std::unique_ptr, + std::unique_ptr, + std::unique_ptr, + std::unique_ptr> { +public: + void read(Context* ctx, std::unique_ptr msg) override; + + folly::Future write(Context* ctx, std::unique_ptr msg) + override; +}; + +using FetchChunkSuccessCallback = std::function)>; + +using FetchChunkFailureCallback = std::function)>; + +/** + * TransportClient sends the messages to the network layer, and handles + * the message callback, timeout, error handling, etc. + */ +class TransportClient { +public: + TransportClient( + std::unique_ptr> client, + std::unique_ptr dispatcher, + Timeout defaultTimeout); + + RpcResponse sendRpcRequestSync(const RpcRequest& request) { + return sendRpcRequestSync(request, defaultTimeout_); + } + + RpcResponse sendRpcRequestSync(const RpcRequest& request, Timeout timeout); + + // Ignore the response, return immediately. + void sendRpcRequestWithoutResponse(const RpcRequest& request); + + void fetchChunkAsync( + const protocol::StreamChunkSlice& streamChunkSlice, + const RpcRequest& request, + FetchChunkSuccessCallback onSuccess, + FetchChunkFailureCallback onFailure); + + bool active() const { + return dispatcher_->isAvailable(); + } + + ~TransportClient() = default; + +private: + std::unique_ptr> client_; + std::unique_ptr dispatcher_; + Timeout defaultTimeout_; +}; + +class MessagePipelineFactory + : public wangle::PipelineFactory { +public: + SerializePipeline::Ptr newPipeline( + std::shared_ptr sock) override; +}; + +class TransportClientFactory { +public: + TransportClientFactory(const std::shared_ptr& conf); + + std::shared_ptr createClient( + const std::string& host, + uint16_t port); + +private: + struct ClientPool { + std::mutex mutex; + std::vector> clients; + }; + + struct Hasher { + size_t operator()(const folly::SocketAddress& lhs) const { + return lhs.hash(); + } + }; + + int numConnectionsPerPeer_; + folly::Synchronized< + std::unordered_map< + folly::SocketAddress, + std::shared_ptr, + Hasher>, + std::mutex> + clientPools_; + Timeout rpcLookupTimeout_; + Timeout connectTimeout_; + int numClientThreads_; + std::shared_ptr clientExecutor_; +}; +} // namespace network +} // namespace celeborn diff --git a/cpp/celeborn/network/tests/CMakeLists.txt b/cpp/celeborn/network/tests/CMakeLists.txt index fdd2c874e22..d33f4410b50 100644 --- a/cpp/celeborn/network/tests/CMakeLists.txt +++ b/cpp/celeborn/network/tests/CMakeLists.txt @@ -17,7 +17,8 @@ add_executable( celeborn_network_test FrameDecoderTest.cpp MessageTest.cpp - MessageDispatcherTest.cpp) + MessageDispatcherTest.cpp + TransportClientTest.cpp) add_test(NAME celeborn_network_test COMMAND celeborn_network_test) @@ -25,6 +26,7 @@ target_link_libraries( celeborn_network_test PRIVATE memory + conf proto protocol network diff --git a/cpp/celeborn/network/tests/MessageDispatcherTest.cpp b/cpp/celeborn/network/tests/MessageDispatcherTest.cpp index 8baf9f8d76a..4d5dbc54111 100644 --- a/cpp/celeborn/network/tests/MessageDispatcherTest.cpp +++ b/cpp/celeborn/network/tests/MessageDispatcherTest.cpp @@ -64,8 +64,8 @@ std::unique_ptr toReadOnlyByteBuffer( } // namespace TEST(MessageDispatcherTest, sendRpcRequestAndReceiveResponse) { - std::unique_ptr sendedMsg; - MockHandler mockHandler(sendedMsg); + std::unique_ptr sentMsg; + MockHandler mockHandler(sentMsg); auto mockPipeline = createMockedPipeline(std::move(mockHandler)); auto dispatcher = std::make_unique(); dispatcher->setPipeline(mockPipeline.get()); @@ -77,11 +77,11 @@ TEST(MessageDispatcherTest, sendRpcRequestAndReceiveResponse) { auto future = dispatcher->sendRpcRequest(std::move(rpcRequest)); EXPECT_FALSE(future.isReady()); - EXPECT_EQ(sendedMsg->type(), Message::RPC_REQUEST); - auto sendedRpcRequest = dynamic_cast(sendedMsg.get()); - EXPECT_EQ(sendedRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); EXPECT_EQ( - sendedRpcRequest->body()->readToString(requestBody.size()), requestBody); + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); const std::string responseBody = "test-response-body"; auto rpcResponse = std::make_unique( @@ -99,8 +99,8 @@ TEST(MessageDispatcherTest, sendRpcRequestAndReceiveResponse) { } TEST(MessageDispatcherTest, sendRpcRequestAndReceiveFailure) { - std::unique_ptr sendedMsg; - MockHandler mockHandler(sendedMsg); + std::unique_ptr sentMsg; + MockHandler mockHandler(sentMsg); auto mockPipeline = createMockedPipeline(std::move(mockHandler)); auto dispatcher = std::make_unique(); dispatcher->setPipeline(mockPipeline.get()); @@ -112,11 +112,11 @@ TEST(MessageDispatcherTest, sendRpcRequestAndReceiveFailure) { auto future = dispatcher->sendRpcRequest(std::move(rpcRequest)); EXPECT_FALSE(future.isReady()); - EXPECT_EQ(sendedMsg->type(), Message::RPC_REQUEST); - auto sendedRpcRequest = dynamic_cast(sendedMsg.get()); - EXPECT_EQ(sendedRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); EXPECT_EQ( - sendedRpcRequest->body()->readToString(requestBody.size()), requestBody); + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); const std::string errorMsg = "test-error-msg"; auto copiedErrorMsg = errorMsg; @@ -128,8 +128,8 @@ TEST(MessageDispatcherTest, sendRpcRequestAndReceiveFailure) { } TEST(MessageDispatcherTest, sendFetchChunkRequestAndReceiveSuccess) { - std::unique_ptr sendedMsg; - MockHandler mockHandler(sendedMsg); + std::unique_ptr sentMsg; + MockHandler mockHandler(sentMsg); auto mockPipeline = createMockedPipeline(std::move(mockHandler)); auto dispatcher = std::make_unique(); dispatcher->setPipeline(mockPipeline.get()); @@ -143,11 +143,11 @@ TEST(MessageDispatcherTest, sendFetchChunkRequestAndReceiveSuccess) { streamChunkSlice, std::move(rpcRequest)); EXPECT_FALSE(future.isReady()); - EXPECT_EQ(sendedMsg->type(), Message::RPC_REQUEST); - auto sendedRpcRequest = dynamic_cast(sendedMsg.get()); - EXPECT_EQ(sendedRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); EXPECT_EQ( - sendedRpcRequest->body()->readToString(requestBody.size()), requestBody); + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); const std::string chunkFetchSuccessBody = "test-chunk-fetch-success-body"; auto chunkFetchSuccess = std::make_unique( @@ -169,8 +169,8 @@ TEST(MessageDispatcherTest, sendFetchChunkRequestAndReceiveSuccess) { } TEST(MessageDispatcherTest, sendFetchChunkRequestAndReceiveFailure) { - std::unique_ptr sendedMsg; - MockHandler mockHandler(sendedMsg); + std::unique_ptr sentMsg; + MockHandler mockHandler(sentMsg); auto mockPipeline = createMockedPipeline(std::move(mockHandler)); auto dispatcher = std::make_unique(); dispatcher->setPipeline(mockPipeline.get()); @@ -184,11 +184,11 @@ TEST(MessageDispatcherTest, sendFetchChunkRequestAndReceiveFailure) { streamChunkSlice, std::move(rpcRequest)); EXPECT_FALSE(future.isReady()); - EXPECT_EQ(sendedMsg->type(), Message::RPC_REQUEST); - auto sendedRpcRequest = dynamic_cast(sendedMsg.get()); - EXPECT_EQ(sendedRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); EXPECT_EQ( - sendedRpcRequest->body()->readToString(requestBody.size()), requestBody); + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); const std::string errorMsg = "test-error-msg"; auto copiedErrorMsg = errorMsg; diff --git a/cpp/celeborn/network/tests/TransportClientTest.cpp b/cpp/celeborn/network/tests/TransportClientTest.cpp new file mode 100644 index 00000000000..86b6467e801 --- /dev/null +++ b/cpp/celeborn/network/tests/TransportClientTest.cpp @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +#include "celeborn/network/TransportClient.h" + +using namespace celeborn; +using namespace celeborn::network; + +namespace { +using MS = std::chrono::milliseconds; +class MockDispatcher : public MessageDispatcher { + public: + folly::Future> sendRpcRequest( + std::unique_ptr toSendMsg) override { + sentMsg_ = std::move(toSendMsg); + msgPromise_ = MsgPromise(); + return msgPromise_.getFuture(); + } + + void sendRpcRequestWithoutResponse( + std::unique_ptr toSendMsg) override { + sentMsg_ = std::move(toSendMsg); + } + + folly::Future> sendFetchChunkRequest( + const protocol::StreamChunkSlice& streamChunkSlice, + std::unique_ptr toSendMsg) override { + sentMsg_ = std::move(toSendMsg); + msgPromise_ = MsgPromise(); + return msgPromise_.getFuture(); + } + + std::unique_ptr getSentMsg() { + return std::move(sentMsg_); + } + + void receiveMsg(std::unique_ptr msg) { + msgPromise_.setValue(std::move(msg)); + } + + private: + using MsgPromise = folly::Promise>; + std::unique_ptr sentMsg_; + MsgPromise msgPromise_; +}; + +std::unique_ptr toReadOnlyByteBuffer( + const std::string& content) { + auto buffer = memory::ByteBuffer::createWriteOnly(content.size()); + buffer->writeFromString(content); + return memory::ByteBuffer::toReadOnly(std::move(buffer)); +} +} // namespace + +TEST(TransportClientTest, sendRpcRequestSync) { + auto mockDispatcher = std::make_unique(); + auto rawMockDispatcher = mockDispatcher.get(); + const auto timeoutInterval = MS(10000); + const auto sleepInterval = MS(100); + TransportClient client(nullptr, std::move(mockDispatcher), timeoutInterval); + + const long requestId = 1001; + const std::string requestBody = "test-request-body"; + auto rpcRequest = std::make_unique( + requestId, toReadOnlyByteBuffer(requestBody)); + std::unique_ptr receivedRpcResponse; + std::thread syncThread([&]() { + auto responseMsg = client.sendRpcRequestSync(*rpcRequest, timeoutInterval); + receivedRpcResponse = std::make_unique(responseMsg); + }); + std::this_thread::sleep_for(sleepInterval); + + auto sentMsg = rawMockDispatcher->getSentMsg(); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ( + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); + + // The response is not received yet. + EXPECT_FALSE(receivedRpcResponse); + const std::string responseBody = "test-response-body"; + auto rpcResponse = std::make_unique( + requestId, toReadOnlyByteBuffer(responseBody)); + rawMockDispatcher->receiveMsg(std::move(rpcResponse)); + + syncThread.join(); + // The response is received. + EXPECT_TRUE(receivedRpcResponse); + EXPECT_EQ(receivedRpcResponse->body()->remainingSize(), responseBody.size()); + EXPECT_EQ( + receivedRpcResponse->body()->readToString(responseBody.size()), + responseBody); +} + +TEST(TransportClientTest, sendRpcRequestSyncTimeout) { + auto mockDispatcher = std::make_unique(); + auto rawMockDispatcher = mockDispatcher.get(); + const auto timeoutInterval = MS(200); + const auto sleepInterval = MS(100); + TransportClient client(nullptr, std::move(mockDispatcher), timeoutInterval); + + const long requestId = 1001; + const std::string requestBody = "test-request-body"; + auto rpcRequest = std::make_unique( + requestId, toReadOnlyByteBuffer(requestBody)); + std::unique_ptr receivedRpcResponse; + bool timeoutHappened = false; + std::thread syncThread([&]() { + try { + auto responseMsg = + client.sendRpcRequestSync(*rpcRequest, timeoutInterval); + receivedRpcResponse = std::make_unique(responseMsg); + } catch (std::exception e) { + timeoutHappened = true; + } + }); + std::this_thread::sleep_for(sleepInterval); + + auto sentMsg = rawMockDispatcher->getSentMsg(); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ( + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); + + // The response is not received yet. + EXPECT_FALSE(receivedRpcResponse); + + syncThread.join(); + // Not response received, should be timeout. + EXPECT_FALSE(receivedRpcResponse); + EXPECT_TRUE(timeoutHappened); +} + +TEST(TransportClientTest, sendRpcRequestWithoutResponse) { + auto mockDispatcher = std::make_unique(); + auto rawMockDispatcher = mockDispatcher.get(); + const auto timeoutInterval = MS(10000); + TransportClient client(nullptr, std::move(mockDispatcher), timeoutInterval); + + const long requestId = 1001; + const std::string requestBody = "test-request-body"; + auto rpcRequest = std::make_unique( + requestId, toReadOnlyByteBuffer(requestBody)); + client.sendRpcRequestWithoutResponse(*rpcRequest); + + auto sentMsg = rawMockDispatcher->getSentMsg(); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ( + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); +} + +TEST(TransportClientTest, fetchChunkAsyncSuccess) { + auto mockDispatcher = std::make_unique(); + auto rawMockDispatcher = mockDispatcher.get(); + TransportClient client(nullptr, std::move(mockDispatcher), MS(10000)); + + const protocol::StreamChunkSlice streamChunkSlice{1, 2, 3, 4}; + const long requestId = 1001; + const std::string requestBody = "test-request-body"; + auto rpcRequest = std::make_unique( + requestId, toReadOnlyByteBuffer(requestBody)); + protocol::StreamChunkSlice onSuccessStreamChunkSlice; + std::unique_ptr onSuccessBuffer; + FetchChunkSuccessCallback onSuccess = + [&](protocol::StreamChunkSlice slice, std::unique_ptr buffer) { + onSuccessStreamChunkSlice = slice; + onSuccessBuffer = std::move(buffer); + }; + protocol::StreamChunkSlice onFailureStreamChunkSlice; + std::unique_ptr onFailureException; + FetchChunkFailureCallback onFailure = + [&](protocol::StreamChunkSlice slice, std::unique_ptr exception) { + onFailureStreamChunkSlice = slice; + onFailureException = std::move(exception); + }; + + client.fetchChunkAsync(streamChunkSlice, *rpcRequest, onSuccess, onFailure); + auto sentMsg = rawMockDispatcher->getSentMsg(); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ( + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); + // The response is not received yet. + EXPECT_FALSE(onSuccessBuffer); + + const std::string responseBody = "test-response-body"; + auto chunkFetchSuccess = std::make_unique( + streamChunkSlice, toReadOnlyByteBuffer(responseBody)); + rawMockDispatcher->receiveMsg(std::move(chunkFetchSuccess)); + + // The response is received. + EXPECT_TRUE(onSuccessBuffer); + EXPECT_FALSE(onFailureException); + EXPECT_EQ(onSuccessBuffer->remainingSize(), responseBody.size()); + EXPECT_EQ(onSuccessBuffer->readToString(responseBody.size()), responseBody); +} + +TEST(TransportClientTest, fetchChunkAsyncFailure) { + auto mockDispatcher = std::make_unique(); + auto rawMockDispatcher = mockDispatcher.get(); + TransportClient client(nullptr, std::move(mockDispatcher), MS(10000)); + + const protocol::StreamChunkSlice streamChunkSlice{1, 2, 3, 4}; + const long requestId = 1001; + const std::string requestBody = "test-request-body"; + auto rpcRequest = std::make_unique( + requestId, toReadOnlyByteBuffer(requestBody)); + protocol::StreamChunkSlice onSuccessStreamChunkSlice; + std::unique_ptr onSuccessBuffer; + FetchChunkSuccessCallback onSuccess = + [&](protocol::StreamChunkSlice slice, std::unique_ptr buffer) { + onSuccessStreamChunkSlice = slice; + onSuccessBuffer = std::move(buffer); + }; + protocol::StreamChunkSlice onFailureStreamChunkSlice; + std::unique_ptr onFailureException; + FetchChunkFailureCallback onFailure = + [&](protocol::StreamChunkSlice slice, std::unique_ptr exception) { + onFailureStreamChunkSlice = slice; + onFailureException = std::move(exception); + }; + + client.fetchChunkAsync(streamChunkSlice, *rpcRequest, onSuccess, onFailure); + auto sentMsg = rawMockDispatcher->getSentMsg(); + EXPECT_EQ(sentMsg->type(), Message::RPC_REQUEST); + auto sentRpcRequest = dynamic_cast(sentMsg.get()); + EXPECT_EQ(sentRpcRequest->body()->remainingSize(), requestBody.size()); + EXPECT_EQ( + sentRpcRequest->body()->readToString(requestBody.size()), requestBody); + // The response is not received yet. + EXPECT_FALSE(onSuccessBuffer); + + auto chunkFetchFailure = std::make_unique( + streamChunkSlice, "test-error-string"); + rawMockDispatcher->receiveMsg(std::move(chunkFetchFailure)); + + // The failure is received. + EXPECT_TRUE(onFailureException); + EXPECT_FALSE(onSuccessBuffer); +} diff --git a/dev/deps/dependencies-client-flink-1.16 b/dev/deps/dependencies-client-flink-1.16 index 4f786dd8046..c303954405c 100644 --- a/dev/deps/dependencies-client-flink-1.16 +++ b/dev/deps/dependencies-client-flink-1.16 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-flink-1.17 b/dev/deps/dependencies-client-flink-1.17 index 4f786dd8046..c303954405c 100644 --- a/dev/deps/dependencies-client-flink-1.17 +++ b/dev/deps/dependencies-client-flink-1.17 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-flink-1.18 b/dev/deps/dependencies-client-flink-1.18 index 4f786dd8046..c303954405c 100644 --- a/dev/deps/dependencies-client-flink-1.18 +++ b/dev/deps/dependencies-client-flink-1.18 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-flink-1.19 b/dev/deps/dependencies-client-flink-1.19 index 4f786dd8046..c303954405c 100644 --- a/dev/deps/dependencies-client-flink-1.19 +++ b/dev/deps/dependencies-client-flink-1.19 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-flink-1.20 b/dev/deps/dependencies-client-flink-1.20 index 4f786dd8046..c303954405c 100644 --- a/dev/deps/dependencies-client-flink-1.20 +++ b/dev/deps/dependencies-client-flink-1.20 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-mr b/dev/deps/dependencies-client-mr index fdeae28e455..340fd7b99c1 100644 --- a/dev/deps/dependencies-client-mr +++ b/dev/deps/dependencies-client-mr @@ -140,40 +140,40 @@ metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar mssql-jdbc/6.2.1.jre7//mssql-jdbc-6.2.1.jre7.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar netty/3.10.6.Final//netty-3.10.6.Final.jar nimbus-jose-jwt/9.8.1//nimbus-jose-jwt-9.8.1.jar okhttp/4.9.3//okhttp-4.9.3.jar diff --git a/dev/deps/dependencies-client-spark-2.4 b/dev/deps/dependencies-client-spark-2.4 index b8a3762d02f..9a98693e5d0 100644 --- a/dev/deps/dependencies-client-spark-2.4 +++ b/dev/deps/dependencies-client-spark-2.4 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.11.12//scala-library-2.11.12.jar diff --git a/dev/deps/dependencies-client-spark-3.0 b/dev/deps/dependencies-client-spark-3.0 index ec7b0bdd9d0..709ee703569 100644 --- a/dev/deps/dependencies-client-spark-3.0 +++ b/dev/deps/dependencies-client-spark-3.0 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.10//scala-library-2.12.10.jar diff --git a/dev/deps/dependencies-client-spark-3.1 b/dev/deps/dependencies-client-spark-3.1 index efb0b4ae653..bcc5e047135 100644 --- a/dev/deps/dependencies-client-spark-3.1 +++ b/dev/deps/dependencies-client-spark-3.1 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.10//scala-library-2.12.10.jar diff --git a/dev/deps/dependencies-client-spark-3.2 b/dev/deps/dependencies-client-spark-3.2 index 7bab9664cf9..1abf62bd92f 100644 --- a/dev/deps/dependencies-client-spark-3.2 +++ b/dev/deps/dependencies-client-spark-3.2 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.15//scala-library-2.12.15.jar diff --git a/dev/deps/dependencies-client-spark-3.3 b/dev/deps/dependencies-client-spark-3.3 index 6b3beffecb0..15aa9a33e81 100644 --- a/dev/deps/dependencies-client-spark-3.3 +++ b/dev/deps/dependencies-client-spark-3.3 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.15//scala-library-2.12.15.jar diff --git a/dev/deps/dependencies-client-spark-3.4 b/dev/deps/dependencies-client-spark-3.4 index 410299b2d39..a7cbf19230b 100644 --- a/dev/deps/dependencies-client-spark-3.4 +++ b/dev/deps/dependencies-client-spark-3.4 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.17//scala-library-2.12.17.jar diff --git a/dev/deps/dependencies-client-spark-3.5 b/dev/deps/dependencies-client-spark-3.5 index 51f40c29e2b..3b40450e32d 100644 --- a/dev/deps/dependencies-client-spark-3.5 +++ b/dev/deps/dependencies-client-spark-3.5 @@ -37,40 +37,40 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.25.5//protobuf-java-3.25.5.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/dev/deps/dependencies-client-tez b/dev/deps/dependencies-client-tez index bf503d114fb..8e99c00a511 100644 --- a/dev/deps/dependencies-client-tez +++ b/dev/deps/dependencies-client-tez @@ -112,42 +112,42 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/4.2.25//metrics-core-4.2.25.jar metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar netty-reactive-streams/2.0.4//netty-reactive-streams-2.0.4.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar netty-transport-native-epoll/4.1.63.Final//netty-transport-native-epoll-4.1.63.Final.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar nimbus-jose-jwt/9.8.1//nimbus-jose-jwt-9.8.1.jar okhttp/2.7.5//okhttp-2.7.5.jar okio/1.6.0//okio-1.6.0.jar diff --git a/dev/deps/dependencies-server b/dev/deps/dependencies-server index 1a34f83a0e1..a65ac688fee 100644 --- a/dev/deps/dependencies-server +++ b/dev/deps/dependencies-server @@ -84,40 +84,40 @@ metrics-graphite/4.2.25//metrics-graphite-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar mimepull/1.9.15//mimepull-1.9.15.jar mybatis/3.5.15//mybatis-3.5.15.jar -netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar -netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar -netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar -netty-codec-haproxy/4.1.115.Final//netty-codec-haproxy-4.1.115.Final.jar -netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar -netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar -netty-codec-memcache/4.1.115.Final//netty-codec-memcache-4.1.115.Final.jar -netty-codec-mqtt/4.1.115.Final//netty-codec-mqtt-4.1.115.Final.jar -netty-codec-redis/4.1.115.Final//netty-codec-redis-4.1.115.Final.jar -netty-codec-smtp/4.1.115.Final//netty-codec-smtp-4.1.115.Final.jar -netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar -netty-codec-stomp/4.1.115.Final//netty-codec-stomp-4.1.115.Final.jar -netty-codec-xml/4.1.115.Final//netty-codec-xml-4.1.115.Final.jar -netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar -netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar -netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar -netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar -netty-resolver-dns-classes-macos/4.1.115.Final//netty-resolver-dns-classes-macos-4.1.115.Final.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar -netty-resolver-dns-native-macos/4.1.115.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar -netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar -netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar -netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar -netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar -netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar -netty-transport-rxtx/4.1.115.Final//netty-transport-rxtx-4.1.115.Final.jar -netty-transport-sctp/4.1.115.Final//netty-transport-sctp-4.1.115.Final.jar -netty-transport-udt/4.1.115.Final//netty-transport-udt-4.1.115.Final.jar -netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar +netty-all/4.1.118.Final//netty-all-4.1.118.Final.jar +netty-buffer/4.1.118.Final//netty-buffer-4.1.118.Final.jar +netty-codec-dns/4.1.118.Final//netty-codec-dns-4.1.118.Final.jar +netty-codec-haproxy/4.1.118.Final//netty-codec-haproxy-4.1.118.Final.jar +netty-codec-http/4.1.118.Final//netty-codec-http-4.1.118.Final.jar +netty-codec-http2/4.1.118.Final//netty-codec-http2-4.1.118.Final.jar +netty-codec-memcache/4.1.118.Final//netty-codec-memcache-4.1.118.Final.jar +netty-codec-mqtt/4.1.118.Final//netty-codec-mqtt-4.1.118.Final.jar +netty-codec-redis/4.1.118.Final//netty-codec-redis-4.1.118.Final.jar +netty-codec-smtp/4.1.118.Final//netty-codec-smtp-4.1.118.Final.jar +netty-codec-socks/4.1.118.Final//netty-codec-socks-4.1.118.Final.jar +netty-codec-stomp/4.1.118.Final//netty-codec-stomp-4.1.118.Final.jar +netty-codec-xml/4.1.118.Final//netty-codec-xml-4.1.118.Final.jar +netty-codec/4.1.118.Final//netty-codec-4.1.118.Final.jar +netty-common/4.1.118.Final//netty-common-4.1.118.Final.jar +netty-handler-proxy/4.1.118.Final//netty-handler-proxy-4.1.118.Final.jar +netty-handler/4.1.118.Final//netty-handler-4.1.118.Final.jar +netty-resolver-dns-classes-macos/4.1.118.Final//netty-resolver-dns-classes-macos-4.1.118.Final.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-aarch_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar +netty-resolver-dns-native-macos/4.1.118.Final/osx-x86_64/netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar +netty-resolver-dns/4.1.118.Final//netty-resolver-dns-4.1.118.Final.jar +netty-resolver/4.1.118.Final//netty-resolver-4.1.118.Final.jar +netty-transport-classes-epoll/4.1.118.Final//netty-transport-classes-epoll-4.1.118.Final.jar +netty-transport-classes-kqueue/4.1.118.Final//netty-transport-classes-kqueue-4.1.118.Final.jar +netty-transport-native-epoll/4.1.118.Final/linux-aarch_64/netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.118.Final/linux-riscv64/netty-transport-native-epoll-4.1.118.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.118.Final/linux-x86_64/netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.118.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.118.Final/osx-x86_64/netty-transport-native-kqueue-4.1.118.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.118.Final//netty-transport-native-unix-common-4.1.118.Final.jar +netty-transport-rxtx/4.1.118.Final//netty-transport-rxtx-4.1.118.Final.jar +netty-transport-sctp/4.1.118.Final//netty-transport-sctp-4.1.118.Final.jar +netty-transport-udt/4.1.118.Final//netty-transport-udt-4.1.118.Final.jar +netty-transport/4.1.118.Final//netty-transport-4.1.118.Final.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar picocli/4.7.6//picocli-4.7.6.jar diff --git a/docs/configuration/client.md b/docs/configuration/client.md index 197dbb7908e..d9e1a700c77 100644 --- a/docs/configuration/client.md +++ b/docs/configuration/client.md @@ -81,10 +81,11 @@ license: | | celeborn.client.rpc.cache.size | 256 | false | The max cache items count for rpc cache. | 0.3.0 | celeborn.rpc.cache.size | | celeborn.client.rpc.commitFiles.askTimeout | <value of celeborn.rpc.askTimeout> | false | Timeout for CommitHandler commit files. | 0.4.1 | | | celeborn.client.rpc.getReducerFileGroup.askTimeout | <value of celeborn.rpc.askTimeout> | false | Timeout for ask operations during getting reducer file group information. During this process, there are `celeborn.client.requestCommitFiles.maxRetries` times for retry opportunities for committing files and 1 times for releasing slots request. User can customize this value according to your setting. | 0.2.0 | | -| celeborn.client.rpc.maxRetries | 3 | false | Max RPC retry times in LifecycleManager. | 0.3.2 | | +| celeborn.client.rpc.maxRetries | 3 | false | Max RPC retry times in client. | 0.3.2 | | | celeborn.client.rpc.registerShuffle.askTimeout | <value of celeborn.rpc.askTimeout> | false | Timeout for ask operations during register shuffle. During this process, there are two times for retry opportunities for requesting slots, one request for establishing a connection with Worker and `celeborn.client.reserveSlots.maxRetries` times for retry opportunities for reserving slots. User can customize this value according to your setting. | 0.3.0 | celeborn.rpc.registerShuffle.askTimeout | | celeborn.client.rpc.requestPartition.askTimeout | <value of celeborn.rpc.askTimeout> | false | Timeout for ask operations during requesting change partition location, such as reviving or splitting partition. During this process, there are `celeborn.client.reserveSlots.maxRetries` times for retry opportunities for reserving slots. User can customize this value according to your setting. | 0.2.0 | | | celeborn.client.rpc.reserveSlots.askTimeout | <value of celeborn.rpc.askTimeout> | false | Timeout for LifecycleManager request reserve slots. | 0.3.0 | | +| celeborn.client.rpc.retryWait | 1s | false | Client-specified time to wait before next retry on RpcTimeoutException. | 0.6.0 | | | celeborn.client.rpc.shared.threads | 16 | false | Number of shared rpc threads in LifecycleManager. | 0.3.2 | | | celeborn.client.shuffle.batchHandleChangePartition.interval | 100ms | false | Interval for LifecycleManager to schedule handling change partition requests in batch. | 0.3.0 | celeborn.shuffle.batchHandleChangePartition.interval | | celeborn.client.shuffle.batchHandleChangePartition.partitionBuckets | 256 | false | Max number of change partition requests which can be concurrently processed. | 0.5.0 | | diff --git a/docs/configuration/network.md b/docs/configuration/network.md index c958451ba05..50b56c17637 100644 --- a/docs/configuration/network.md +++ b/docs/configuration/network.md @@ -56,6 +56,7 @@ license: | | celeborn.rpc.inbox.capacity | 0 | false | Specifies size of the in memory bounded capacity. | 0.5.0 | | | celeborn.rpc.io.threads | <undefined> | false | Netty IO thread number of NettyRpcEnv to handle RPC request. The default threads number is the number of runtime available processors. | 0.2.0 | | | celeborn.rpc.lookupTimeout | 30s | false | Timeout for RPC lookup operations. | 0.2.0 | | +| celeborn.rpc.retryWait | 1s | false | Time to wait before next retry on RpcTimeoutException. | 0.6.0 | | | celeborn.rpc.slow.interval | <undefined> | false | min interval (ms) for RPC framework to log slow RPC | 0.6.0 | | | celeborn.rpc.slow.threshold | 1s | false | threshold for RPC framework to log slow RPC | 0.6.0 | | | celeborn.shuffle.io.maxChunksBeingTransferred | <undefined> | false | The max number of chunks allowed to be transferred at the same time on shuffle service. Note that new incoming connections will be closed when the max number is hit. The client will retry according to the shuffle retry configs (see `celeborn..io.maxRetries` and `celeborn..io.retryWait`), if those limits are reached the task will fail with fetch failure. | 0.2.0 | | diff --git a/pom.xml b/pom.xml index 0ad395a0d1d..1c8aeee1cc9 100644 --- a/pom.xml +++ b/pom.xml @@ -92,7 +92,7 @@ 1.8.0 4.11.0 1.17.14 - 4.1.115.Final + 4.1.118.Final 1.77 3.25.5 3.1.3 diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index 6868d532f71..f2f61de4fab 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -59,7 +59,7 @@ object Dependencies { val jdkToolsVersion = "0.1" val metricsVersion = "4.2.25" val mockitoVersion = "4.11.0" - val nettyVersion = "4.1.115.Final" + val nettyVersion = "4.1.118.Final" val ratisVersion = "3.1.3" val roaringBitmapVersion = "1.0.6" val rocksdbJniVersion = "9.5.2"