diff --git a/.gitattributes b/.gitattributes index e87caf595b..23353cbc51 100644 --- a/.gitattributes +++ b/.gitattributes @@ -31,3 +31,6 @@ assets/img/* export-ignore *.scala text eol=lf *.xml text eol=lf *.py text eol=lf +common/src/test/resources/ssl/generate_certs.sh text +common/src/test/resources/ssl/* -text +worker/src/test/resources/ssl/* -text diff --git a/DISCLAIMER b/DISCLAIMER deleted file mode 100644 index 0e5e17ddeb..0000000000 --- a/DISCLAIMER +++ /dev/null @@ -1,10 +0,0 @@ -Apache Celeborn (Incubating) is an effort undergoing incubation at the Apache -Software Foundation (ASF), sponsored by the Apache Incubator PMC. - -Incubation is required of all newly accepted projects until a further review -indicates that the infrastructure, communications, and decision making process -have stabilized in a manner consistent with other successful ASF projects. - -While incubation status is not necessarily a reflection of the completeness -or stability of the code, it does indicate that the project has yet to be -fully endorsed by the ASF. diff --git a/LICENSE b/LICENSE index 30b295e64f..d5f68e4d72 100644 --- a/LICENSE +++ b/LICENSE @@ -212,9 +212,16 @@ Apache License 2.0 Apache Spark ./client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java ./client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java +./common/src/main/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeader.java +./common/src/main/java/org/apache/celeborn/common/network/protocol/SslMessageEncoder.java +./common/src/main/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManager.java +./common/src/main/java/org/apache/celeborn/common/network/util/NettyLogger.java ./common/src/main/java/org/apache/celeborn/common/unsafe/Platform.java ./common/src/main/java/org/apache/celeborn/common/util/JavaUtils.java ./common/src/main/scala/org/apache/celeborn/common/util/SignalUtils.scala +./common/src/test/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeaderSuiteJ.java +./common/src/test/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManagerSuiteJ.java +./common/src/test/java/org/apache/celeborn/common/network/ssl/SslSampleConfigs.java ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/DB.java ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/DBIterator.java ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/LevelDB.java @@ -226,6 +233,7 @@ Apache Spark ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/LevelDBIterator.java ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/RocksDB.java ./worker/src/main/java/org/apache/celeborn/service/deploy/worker/shuffledb/RocksDBProvider.java +./worker/src/main/scala/org/apache/celeborn/service/deploy/worker/profiler/JVMProfiler.scala Apache Kyuubi ./common/src/main/java/org/apache/celeborn/reflect/DynClasses.java diff --git a/LICENSE-binary b/LICENSE-binary index a33986495a..0341413170 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -248,6 +248,11 @@ io.netty:netty-transport-native-unix-common io.netty:netty-transport-rxtx io.netty:netty-transport-sctp io.netty:netty-transport-udt +io.swagger.core.v3:swagger-annotations +io.swagger.core.v3:swagger-core +io.swagger.core.v3:swagger-integration +io.swagger.core.v3:swagger-jaxrs2 +io.swagger.core.v3:swagger-models org.apache.commons:commons-crypto org.apache.commons:commons-lang3 org.apache.hadoop:hadoop-client-api @@ -267,6 +272,13 @@ org.apache.ratis:ratis-server org.apache.ratis:ratis-server-api org.apache.ratis:ratis-shell org.apache.ratis:ratis-thirdparty-misc +org.eclipse.jetty:jetty-http +org.eclipse.jetty:jetty-io +org.eclipse.jetty:jetty-security +org.eclipse.jetty:jetty-server +org.eclipse.jetty:jetty-servlet +org.eclipse.jetty:jetty-util-ajax +org.eclipse.jetty:jetty-util org.javassist:javassist org.reflections:reflections org.roaringbitmap:RoaringBitmap @@ -275,6 +287,7 @@ org.rocksdb:rocksdbjni org.scala-lang:scala-library org.scala-lang:scala-reflect org.slf4j:jcl-over-slf4j +org.webjars:swagger-ui org.xerial.snappy:snappy-java org.yaml:snakeyaml @@ -308,3 +321,15 @@ org.slf4j:slf4j-api ------------ See licenses/LICENSE-javassist.txt for detail. org.javassist:javassist + +Eclipse Public License (EPL) 2.0 +-------------------------------- +jakarta.annotation:jakarta.annotation-api +jakarta.servlet:jakarta.servlet-api +jakarta.ws.rs:jakarta.ws.rs-api +org.glassfish.hk2:hk2-api +org.glassfish.hk2:hk2-locator +org.glassfish.hk2:hk2-utils +org.glassfish.hk2.external:aopalliance-repackaged +org.glassfish.hk2.external:jakarta.inject +org.glassfish.hk2:osgi-resource-locator diff --git a/NOTICE b/NOTICE index 34ec3f608e..b63ca7b199 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/NOTICE-binary b/NOTICE-binary index 4a3bb196ff..1c9f1884bc 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/README.md b/README.md index a0b7fb4c0c..a720805f2e 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,6 @@ -# Apache Celeborn (Incubating) +# Apache Celeborn -[![Celeborn CI](https://github.com/apache/incubator-celeborn/actions/workflows/maven.yml/badge.svg)](https://github.com/apache/incubator-celeborn/actions/workflows/maven.yml) +[![Celeborn CI](https://github.com/apache/celeborn/actions/workflows/maven.yml/badge.svg)](https://github.com/apache/celeborn/actions/workflows/maven.yml) Celeborn (/หˆkelษ™bษ”:n/) is dedicated to improving the efficiency and elasticity of different map-reduce engines and provides an elastic, high-efficient management service for intermediate data including shuffle data, spilled data, result data, etc. Currently, Celeborn is focusing on shuffle data. @@ -79,6 +79,15 @@ To compile the client for Spark 2.4 with Scala 2.12, please use the following co ./build/make-distribution.sh -DskipTests -Pspark-2.4 -Dscala.version=${scala.version} -Dscala.binary.version=2.12 ``` +To compile for Spark 3.5 with Java21, please use the following command +```shell +./build/make-distribution.sh -Pspark-3.5 -Pjdk-21 +``` +```shell +./build/make-distribution.sh --sbt-enabled -Pspark-3.5 -Pjdk-21 +``` + + ### Package Details Build procedure will create a compressed package. @@ -370,7 +379,7 @@ Contact us through the following mailing list. ### Report Issues or Submit Pull Request -If you meet any questions, feel free to file a ๐Ÿ”—[Jira Ticket](https://issues.apache.org/jira/projects/CELEBORN/issues) or connect us and fix it by submitting a ๐Ÿ”—[Pull Request](https://github.com/apache/incubator-celeborn/pulls). +If you meet any questions, feel free to file a ๐Ÿ”—[Jira Ticket](https://issues.apache.org/jira/projects/CELEBORN/issues) or connect us and fix it by submitting a ๐Ÿ”—[Pull Request](https://github.com/apache/celeborn/pulls). | IM | Contact Info | |:---------|:------------------------------------------------------------------------------------------------------------------------------------------| diff --git a/build/make-distribution.sh b/build/make-distribution.sh index cffc6a80f3..5a93331992 100755 --- a/build/make-distribution.sh +++ b/build/make-distribution.sh @@ -306,6 +306,7 @@ if [ "$SBT_ENABLED" == "true" ]; then if [ "$RELEASE" == "true" ]; then sbt_build_client -Pspark-2.4 sbt_build_client -Pspark-3.4 + sbt_build_client -Pspark-3.5 sbt_build_client -Pflink-1.14 sbt_build_client -Pflink-1.15 sbt_build_client -Pflink-1.17 @@ -338,6 +339,7 @@ else build_service build_spark_client -Pspark-2.4 build_spark_client -Pspark-3.4 + build_spark_client -Pspark-3.5 build_flink_client -Pflink-1.14 build_flink_client -Pflink-1.15 build_flink_client -Pflink-1.17 @@ -396,7 +398,6 @@ cp "$PROJECT_DIR/docker/Dockerfile" "$DIST_DIR/docker" cp -r "$PROJECT_DIR/charts" "$DIST_DIR" # Copy license files -cp "$PROJECT_DIR/DISCLAIMER" "$DIST_DIR/DISCLAIMER" if [[ -f $"$PROJECT_DIR/LICENSE-binary" ]]; then cp "$PROJECT_DIR/LICENSE-binary" "$DIST_DIR/LICENSE" cp -r "$PROJECT_DIR/licenses-binary" "$DIST_DIR/licenses" diff --git a/build/release/release.sh b/build/release/release.sh index bd93bf8d14..abea587729 100755 --- a/build/release/release.sh +++ b/build/release/release.sh @@ -56,8 +56,8 @@ fi RELEASE_TAG="v${RELEASE_VERSION}-rc${RELEASE_RC_NO}" -SVN_STAGING_REPO="https://dist.apache.org/repos/dist/dev/incubator/celeborn" -SVN_RELEASE_REPO="https://dist.apache.org/repos/dist/release/incubator/celeborn" +SVN_STAGING_REPO="https://dist.apache.org/repos/dist/dev/celeborn" +SVN_RELEASE_REPO="https://dist.apache.org/repos/dist/release/celeborn" RELEASE_DIR="${PROJECT_DIR}/tmp" SVN_STAGING_DIR="${PROJECT_DIR}/tmp/svn-dev" diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java index c3bc6c2981..e6e920b7de 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java @@ -44,9 +44,6 @@ public abstract class AbstractRemoteShuffleInputGateFactory { /** Number of max concurrent reading channels. */ protected final int numConcurrentReading; - /** Codec used for compression / decompression. */ - protected static final String compressionCodec = "LZ4"; - /** Network buffer size. */ protected final int networkBufferSize; @@ -66,18 +63,8 @@ public abstract class AbstractRemoteShuffleInputGateFactory { public AbstractRemoteShuffleInputGateFactory( CelebornConf conf, NetworkBufferPool networkBufferPool, int networkBufferSize) { this.celebornConf = conf; - long configuredMemorySize = celebornConf.clientFlinkMemoryPerInputGate(); - long minConfiguredMemorySize = celebornConf.clientFlinkMemoryPerInputGateMin(); - if (configuredMemorySize < minConfiguredMemorySize) { - throw new IllegalArgumentException( - String.format( - "Insufficient network memory per input gate, please increase %s to at " + "least %s.", - CelebornConf.CLIENT_MEMORY_PER_INPUT_GATE().key(), - celebornConf.clientFlinkMemoryPerInputGateMin())); - } - - this.numBuffersPerGate = Utils.checkedDownCast(configuredMemorySize / networkBufferSize); - this.supportFloatingBuffers = celebornConf.clientFlinkInputGateSupportFloatingBuffer(); + this.numBuffersPerGate = + Utils.checkedDownCast(celebornConf.clientFlinkMemoryPerInputGate() / networkBufferSize); if (numBuffersPerGate < MIN_BUFFERS_PER_GATE) { throw new IllegalArgumentException( String.format( @@ -86,7 +73,7 @@ public AbstractRemoteShuffleInputGateFactory( CelebornConf.CLIENT_MEMORY_PER_INPUT_GATE().key(), networkBufferSize * MIN_BUFFERS_PER_GATE)); } - + this.supportFloatingBuffers = celebornConf.clientFlinkInputGateSupportFloatingBuffer(); this.networkBufferSize = networkBufferSize; this.numConcurrentReading = celebornConf.clientFlinkNumConcurrentReading(); this.networkBufferPool = networkBufferPool; @@ -104,7 +91,7 @@ public IndexedInputGate create( SupplierWithException bufferPoolFactory = createBufferPoolFactory(networkBufferPool, numBuffersPerGate, supportFloatingBuffers); BufferDecompressor bufferDecompressor = - new BufferDecompressor(networkBufferSize, compressionCodec); + new BufferDecompressor(networkBufferSize, celebornConf.shuffleCompressionCodec().name()); return createInputGate(owningTaskName, gateIndex, igdd, bufferPoolFactory, bufferDecompressor); } diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java index 35fbf324b5..e22d6ae479 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java @@ -71,18 +71,9 @@ public AbstractRemoteShuffleResultPartitionFactory( ResultPartitionManager partitionManager, BufferPoolFactory bufferPoolFactory, int networkBufferSize) { - long configuredMemorySize = celebornConf.clientFlinkMemoryPerResultPartition(); - long minConfiguredMemorySize = celebornConf.clientFlinkMemoryPerResultPartitionMin(); - if (configuredMemorySize < minConfiguredMemorySize) { - throw new IllegalArgumentException( - String.format( - "Insufficient network memory per result partition, please increase %s " - + "to at least %s.", - CelebornConf.CLIENT_MEMORY_PER_RESULT_PARTITION().key(), minConfiguredMemorySize)); - } - - this.numBuffersPerPartition = Utils.checkedDownCast(configuredMemorySize / networkBufferSize); - this.supportFloatingBuffers = celebornConf.clientFlinkResultPartitionSupportFloatingBuffer(); + this.numBuffersPerPartition = + Utils.checkedDownCast( + celebornConf.clientFlinkMemoryPerResultPartition() / networkBufferSize); if (numBuffersPerPartition < MIN_BUFFERS_PER_PARTITION) { throw new IllegalArgumentException( String.format( @@ -91,11 +82,10 @@ public AbstractRemoteShuffleResultPartitionFactory( CelebornConf.CLIENT_MEMORY_PER_RESULT_PARTITION().key(), networkBufferSize * MIN_BUFFERS_PER_PARTITION)); } - + this.supportFloatingBuffers = celebornConf.clientFlinkResultPartitionSupportFloatingBuffer(); this.partitionManager = partitionManager; this.bufferPoolFactory = bufferPoolFactory; this.networkBufferSize = networkBufferSize; - this.compressionCodec = celebornConf.shuffleCompressionCodec().name(); } diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/FlinkNettyManagedBuffer.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/FlinkNettyManagedBuffer.java index aeb736a39b..e3add9117f 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/FlinkNettyManagedBuffer.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/FlinkNettyManagedBuffer.java @@ -17,6 +17,7 @@ package org.apache.celeborn.plugin.flink.buffer; +import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -64,4 +65,9 @@ public ManagedBuffer release() { public Object convertToNetty() { return buf.duplicate().retain(); } + + @Override + public Object convertToNettyForSsl() throws IOException { + return buf.duplicate().retain(); + } } diff --git a/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/PluginSideConfSuiteJ.java b/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/PluginSideConfSuiteJ.java index 99a6a83cd5..4518d4a920 100644 --- a/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/PluginSideConfSuiteJ.java +++ b/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/PluginSideConfSuiteJ.java @@ -29,8 +29,6 @@ public class PluginSideConfSuiteJ { public void testCoalesce() { Configuration flinkConf = new Configuration(); CelebornConf celebornConf = FlinkUtils.toCelebornConf(flinkConf); - Assert.assertEquals(8 * 1024 * 1024, celebornConf.clientFlinkMemoryPerResultPartitionMin()); - Assert.assertEquals(8 * 1024 * 1024, celebornConf.clientFlinkMemoryPerInputGateMin()); Assert.assertEquals(Integer.MAX_VALUE, celebornConf.clientFlinkNumConcurrentReading()); Assert.assertEquals(64 * 1024 * 1024, celebornConf.clientFlinkMemoryPerResultPartition()); Assert.assertEquals(32 * 1024 * 1024, celebornConf.clientFlinkMemoryPerInputGate()); @@ -38,8 +36,6 @@ public void testCoalesce() { Assert.assertTrue(celebornConf.clientFlinkDataCompressionEnabled()); Assert.assertEquals("LZ4", celebornConf.shuffleCompressionCodec().name()); - flinkConf.setString("remote-shuffle.job.min.memory-per-partition", "16m"); - flinkConf.setString("remote-shuffle.job.min.memory-per-gate", "17m"); flinkConf.setString("remote-shuffle.job.concurrent-readings-per-gate", "12323"); flinkConf.setString("remote-shuffle.job.memory-per-partition", "1888m"); flinkConf.setString("remote-shuffle.job.memory-per-gate", "176m"); @@ -48,8 +44,6 @@ public void testCoalesce() { flinkConf.setString("remote-shuffle.job.compression.codec", "ZSTD"); celebornConf = FlinkUtils.toCelebornConf(flinkConf); - Assert.assertEquals(16 * 1024 * 1024, celebornConf.clientFlinkMemoryPerResultPartitionMin()); - Assert.assertEquals(17 * 1024 * 1024, celebornConf.clientFlinkMemoryPerInputGateMin()); Assert.assertEquals(12323, celebornConf.clientFlinkNumConcurrentReading()); Assert.assertEquals(1888 * 1024 * 1024, celebornConf.clientFlinkMemoryPerResultPartition()); Assert.assertEquals(176 * 1024 * 1024, celebornConf.clientFlinkMemoryPerInputGate()); diff --git a/client-flink/flink-1.14-shaded/src/main/resources/META-INF/NOTICE b/client-flink/flink-1.14-shaded/src/main/resources/META-INF/NOTICE index 63b5024b0e..43452a38af 100644 --- a/client-flink/flink-1.14-shaded/src/main/resources/META-INF/NOTICE +++ b/client-flink/flink-1.14-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-flink/flink-1.15-shaded/src/main/resources/META-INF/NOTICE b/client-flink/flink-1.15-shaded/src/main/resources/META-INF/NOTICE index 63b5024b0e..43452a38af 100644 --- a/client-flink/flink-1.15-shaded/src/main/resources/META-INF/NOTICE +++ b/client-flink/flink-1.15-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-flink/flink-1.17-shaded/src/main/resources/META-INF/NOTICE b/client-flink/flink-1.17-shaded/src/main/resources/META-INF/NOTICE index 63b5024b0e..43452a38af 100644 --- a/client-flink/flink-1.17-shaded/src/main/resources/META-INF/NOTICE +++ b/client-flink/flink-1.17-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-flink/flink-1.18-shaded/src/main/resources/META-INF/NOTICE b/client-flink/flink-1.18-shaded/src/main/resources/META-INF/NOTICE index 63b5024b0e..43452a38af 100644 --- a/client-flink/flink-1.18-shaded/src/main/resources/META-INF/NOTICE +++ b/client-flink/flink-1.18-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-flink/flink-1.19-shaded/src/main/resources/META-INF/NOTICE b/client-flink/flink-1.19-shaded/src/main/resources/META-INF/NOTICE index 63b5024b0e..43452a38af 100644 --- a/client-flink/flink-1.19-shaded/src/main/resources/META-INF/NOTICE +++ b/client-flink/flink-1.19-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-mr/mr-shaded/src/main/resources/META-INF/NOTICE b/client-mr/mr-shaded/src/main/resources/META-INF/NOTICE index 5b5319639e..9a5437b44e 100644 --- a/client-mr/mr-shaded/src/main/resources/META-INF/NOTICE +++ b/client-mr/mr-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-spark/spark-2-shaded/src/main/resources/META-INF/NOTICE b/client-spark/spark-2-shaded/src/main/resources/META-INF/NOTICE index 1fd47fe3d9..c48952d00d 100644 --- a/client-spark/spark-2-shaded/src/main/resources/META-INF/NOTICE +++ b/client-spark/spark-2-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/client-spark/spark-3-shaded/src/main/resources/META-INF/NOTICE b/client-spark/spark-3-shaded/src/main/resources/META-INF/NOTICE index 1fd47fe3d9..c48952d00d 100644 --- a/client-spark/spark-3-shaded/src/main/resources/META-INF/NOTICE +++ b/client-spark/spark-3-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ -Apache Celeborn (Incubating) +Apache Celeborn Copyright 2022-2024 The Apache Software Foundation. This product includes software developed at diff --git a/common/pom.xml b/common/pom.xml index e9e8a19f08..b7ae4cfc18 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -147,6 +147,17 @@ log4j-1.2-api test + + + org.bouncycastle + bcprov-jdk18on + test + + + org.bouncycastle + bcpkix-jdk18on + test + diff --git a/common/src/main/java/org/apache/celeborn/common/client/MasterClient.java b/common/src/main/java/org/apache/celeborn/common/client/MasterClient.java index c34e1050bf..94e4201a99 100644 --- a/common/src/main/java/org/apache/celeborn/common/client/MasterClient.java +++ b/common/src/main/java/org/apache/celeborn/common/client/MasterClient.java @@ -175,7 +175,10 @@ private boolean shouldRetry(@Nullable RpcEndpointRef oldRef, Throwable e) { // 'CelebornException: Exception thrown in awaitResult' if (e.getCause() instanceof MasterNotLeaderException) { MasterNotLeaderException exception = (MasterNotLeaderException) e.getCause(); - String leaderAddr = exception.getSuggestedLeaderAddress(); + String leaderAddr = + isWorker + ? exception.getSuggestedInternalLeaderAddress() + : exception.getSuggestedLeaderAddress(); if (!leaderAddr.equals(MasterNotLeaderException.LEADER_NOT_PRESENTED)) { setRpcEndpointRef(leaderAddr); } else { diff --git a/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java b/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java index 38198cf786..b613e1041f 100644 --- a/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java +++ b/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java @@ -28,26 +28,42 @@ public class MasterNotLeaderException extends IOException { private static final long serialVersionUID = -2552475565785098271L; private final String leaderPeer; + private final String internalLeaderPeer; public static final String LEADER_NOT_PRESENTED = "leader is not present"; public MasterNotLeaderException( String currentPeer, String suggestedLeaderPeer, @Nullable Throwable cause) { + this(currentPeer, suggestedLeaderPeer, suggestedLeaderPeer, cause); + } + + public MasterNotLeaderException( + String currentPeer, + String suggestedLeaderPeer, + String suggestedInternalLeaderPeer, + @Nullable Throwable cause) { super( String.format( "Master:%s is not the leader.%s%s", currentPeer, currentPeer.equals(suggestedLeaderPeer) ? StringUtils.EMPTY - : String.format(" Suggested leader is Master:%s.", suggestedLeaderPeer), + : String.format( + " Suggested leader is Master:%s (%s).", + suggestedLeaderPeer, suggestedInternalLeaderPeer), cause == null ? StringUtils.EMPTY : String.format(" Exception:%s.", cause.getMessage())), cause); this.leaderPeer = suggestedLeaderPeer; + this.internalLeaderPeer = suggestedInternalLeaderPeer; } public String getSuggestedLeaderAddress() { return leaderPeer; } + + public String getSuggestedInternalLeaderAddress() { + return internalLeaderPeer; + } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java b/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java index 488e0fd04f..ec0d1fd877 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java +++ b/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java @@ -23,6 +23,7 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelPipeline; import io.netty.channel.socket.SocketChannel; import io.netty.handler.timeout.IdleStateHandler; import org.slf4j.Logger; @@ -36,6 +37,7 @@ import org.apache.celeborn.common.network.protocol.MessageEncoder; import org.apache.celeborn.common.network.server.*; import org.apache.celeborn.common.network.util.FrameDecoder; +import org.apache.celeborn.common.network.util.NettyLogger; import org.apache.celeborn.common.network.util.TransportConf; import org.apache.celeborn.common.network.util.TransportFrameDecoder; @@ -55,6 +57,7 @@ public class TransportContext { private static final Logger logger = LoggerFactory.getLogger(TransportContext.class); + private static final NettyLogger nettyLogger = new NettyLogger(); private final TransportConf conf; private final BaseMessageHandler msgHandler; private final ChannelDuplexHandler channelsLimiter; @@ -147,12 +150,15 @@ public TransportChannelHandler initializePipeline( ChannelInboundHandlerAdapter decoder, BaseMessageHandler resolvedMsgHandler) { try { + ChannelPipeline pipeline = channel.pipeline(); + if (nettyLogger.getLoggingHandler() != null) { + pipeline.addLast("loggingHandler", nettyLogger.getLoggingHandler()); + } if (channelsLimiter != null) { - channel.pipeline().addLast("limiter", channelsLimiter); + pipeline.addLast("limiter", channelsLimiter); } TransportChannelHandler channelHandler = createChannelHandler(channel, resolvedMsgHandler); - channel - .pipeline() + pipeline .addLast("encoder", ENCODER) .addLast(FrameDecoder.HANDLER_NAME, decoder) .addLast( diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java index 6af9e43050..5d11e87805 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java @@ -24,6 +24,7 @@ import com.google.common.io.ByteStreams; import io.netty.channel.DefaultFileRegion; +import io.netty.handler.stream.ChunkedStream; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -132,6 +133,12 @@ public Object convertToNetty() throws IOException { } } + @Override + public Object convertToNettyForSsl() throws IOException { + // Cannot use zero-copy with SSL + return new ChunkedStream(createInputStream(), conf.maxSslEncryptedBlockSize()); + } + public File getFile() { return file; } diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/ManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/ManagedBuffer.java index ce320d9d7e..9ab05781bb 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/ManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/ManagedBuffer.java @@ -71,4 +71,15 @@ public abstract class ManagedBuffer { * the caller will be responsible for releasing this new reference. */ public abstract Object convertToNetty() throws IOException; + + /** + * Convert the buffer into a Netty object, used to write the data out with SSL encryption, which + * cannot use {@link io.netty.channel.FileRegion}. The return value is either a {@link + * io.netty.buffer.ByteBuf}, a {@link io.netty.handler.stream.ChunkedStream}, or a {@link + * java.io.InputStream}. + * + *

If this method returns a ByteBuf, then that buffer's reference count will be incremented and + * the caller will be responsible for releasing this new reference. + */ + public abstract Object convertToNettyForSsl() throws IOException; } diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java index 60cf8625b1..0528c8c74a 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java @@ -76,6 +76,11 @@ public Object convertToNetty() throws IOException { return buf.duplicate().retain(); } + @Override + public Object convertToNettyForSsl() throws IOException { + return buf.duplicate().retain(); + } + @Override public String toString() { return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java index b14cb1f8ee..97c31ef2ce 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java @@ -64,6 +64,11 @@ public Object convertToNetty() throws IOException { return Unpooled.wrappedBuffer(buf); } + @Override + public Object convertToNettyForSsl() throws IOException { + return Unpooled.wrappedBuffer(buf); + } + @Override public String toString() { return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) diff --git a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java index f4b62d872f..fd64b6bd03 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java +++ b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java @@ -354,7 +354,8 @@ public String getClientId() { *

Trying to set a different client ID after it's been set will result in an exception. */ public void setClientId(String id) { - Preconditions.checkState(clientId == null, "Client ID has already been set."); + Preconditions.checkState( + clientId == null || clientId.equals(id), "Client ID has already been set."); this.clientId = id; } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeader.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeader.java new file mode 100644 index 0000000000..df2ab1a929 --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeader.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.protocol; + +import java.io.EOFException; +import java.io.InputStream; + +import javax.annotation.Nullable; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.stream.ChunkedInput; +import io.netty.handler.stream.ChunkedStream; + +import org.apache.celeborn.common.network.buffer.ManagedBuffer; + +/** + * A wrapper message that holds two separate pieces (a header and a body). + * + *

The header must be a ByteBuf, while the body can be any InputStream or ChunkedStream Based on + * common/network-common/org.apache.spark.network.protocol.EncryptedMessageWithHeader + */ +public class EncryptedMessageWithHeader implements ChunkedInput { + + @Nullable private final ManagedBuffer managedBuffer; + private final ByteBuf header; + private final int headerLength; + private final Object body; + private final long bodyLength; + private long totalBytesTransferred; + + /** + * Construct a new EncryptedMessageWithHeader. + * + * @param managedBuffer the {@link ManagedBuffer} that the message body came from. This needs to + * be passed in so that the buffer can be freed when this message is deallocated. Ownership of + * the caller's reference to this buffer is transferred to this class, so if the caller wants + * to continue to use the ManagedBuffer in other messages then they will need to call retain() + * on it before passing it to this constructor. + * @param header the message header. + * @param body the message body. + * @param bodyLength the length of the message body, in bytes. + */ + public EncryptedMessageWithHeader( + @Nullable ManagedBuffer managedBuffer, ByteBuf header, Object body, long bodyLength) { + Preconditions.checkArgument( + body instanceof InputStream || body instanceof ChunkedStream, + "Body must be an InputStream or a ChunkedStream."); + this.managedBuffer = managedBuffer; + this.header = header; + this.headerLength = header.readableBytes(); + this.body = body; + this.bodyLength = bodyLength; + this.totalBytesTransferred = 0; + } + + @Override + public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception { + return readChunk(ctx.alloc()); + } + + @Override + public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception { + if (isEndOfInput()) { + return null; + } + + if (totalBytesTransferred < headerLength) { + totalBytesTransferred += headerLength; + return header.retain(); + } else if (body instanceof InputStream) { + InputStream stream = (InputStream) body; + int available = stream.available(); + if (available <= 0) { + available = (int) (length() - totalBytesTransferred); + } else { + available = (int) Math.min(available, length() - totalBytesTransferred); + } + ByteBuf buffer = allocator.buffer(available); + int toRead = Math.min(available, buffer.writableBytes()); + int read = buffer.writeBytes(stream, toRead); + if (read >= 0) { + totalBytesTransferred += read; + return buffer; + } else { + throw new EOFException("Unable to read bytes from InputStream"); + } + } else if (body instanceof ChunkedStream) { + ChunkedStream stream = (ChunkedStream) body; + long old = stream.transferredBytes(); + ByteBuf buffer = stream.readChunk(allocator); + long read = stream.transferredBytes() - old; + if (read >= 0) { + totalBytesTransferred += read; + assert (totalBytesTransferred <= length()); + return buffer; + } else { + throw new EOFException("Unable to read bytes from ChunkedStream"); + } + } else { + return null; + } + } + + @Override + public long length() { + return headerLength + bodyLength; + } + + @Override + public long progress() { + return totalBytesTransferred; + } + + @Override + public boolean isEndOfInput() throws Exception { + return (headerLength + bodyLength) == totalBytesTransferred; + } + + @Override + public void close() throws Exception { + header.release(); + if (managedBuffer != null) { + managedBuffer.release(); + } + if (body instanceof InputStream) { + ((InputStream) body).close(); + } else if (body instanceof ChunkedStream) { + ((ChunkedStream) body).close(); + } + } +} diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/MessageWithHeader.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/MessageWithHeader.java index 21f11d49f8..c2de261289 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/MessageWithHeader.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/MessageWithHeader.java @@ -192,4 +192,9 @@ public boolean release(int decrement) { } return super.release(decrement); } + + @Override + public String toString() { + return "MessageWithHeader [headerLength: " + headerLength + ", bodyLength: " + bodyLength + "]"; + } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/SslMessageEncoder.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/SslMessageEncoder.java new file mode 100644 index 0000000000..508b6a13d7 --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/SslMessageEncoder.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.protocol; + +import java.io.InputStream; +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageEncoder; +import io.netty.handler.stream.ChunkedStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encoder used by the server side to encode secure (SSL) server-to-client responses. This encoder + * is stateless so it is safe to be shared by multiple threads. Based on + * common/network-common/org.apache.spark.network.protocol.SslMessageEncoder + */ +@ChannelHandler.Sharable +public final class SslMessageEncoder extends MessageToMessageEncoder { + + private static final Logger logger = LoggerFactory.getLogger(SslMessageEncoder.class); + public static final SslMessageEncoder INSTANCE = new SslMessageEncoder(); + + private SslMessageEncoder() {} + + /** + * Encodes a Message by invoking its encode() method. For non-data messages, we will add one + * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. + * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the + * data to 'out'. + */ + @Override + public void encode(ChannelHandlerContext ctx, Message in, List out) throws Exception { + Object body = null; + int bodyLength = 0; + + // If the message has a body, take it out... + // For SSL, zero-copy transfer will not work, so we will check if + // the body is an InputStream, and if so, use an EncryptedMessageWithHeader + // to wrap the header+body appropriately (for thread safety). + if (in.body() != null) { + try { + bodyLength = (int) in.body().size(); + body = in.body().convertToNettyForSsl(); + } catch (Exception e) { + in.body().release(); + if (in instanceof ResponseMessage) { + ResponseMessage resp = (ResponseMessage) in; + // Re-encode this message as a failure response. + String error = e.getMessage() != null ? e.getMessage() : "null"; + logger.error( + String.format("Error processing %s for client %s", in, ctx.channel().remoteAddress()), + e); + encode(ctx, resp.createFailureResponse(error), out); + } else { + throw e; + } + return; + } + } + + Message.Type msgType = in.type(); + // message size, message type size, body size, message encoded length + int headerLength = 4 + msgType.encodedLength() + 4 + in.encodedLength(); + ByteBuf header = ctx.alloc().heapBuffer(headerLength); + header.writeInt(in.encodedLength()); + msgType.encode(header); + header.writeInt(bodyLength); + in.encode(header); + assert header.writableBytes() == 0; + + if (body != null && bodyLength > 0) { + if (body instanceof ByteBuf) { + out.add(Unpooled.wrappedBuffer(header, (ByteBuf) body)); + } else if (body instanceof InputStream || body instanceof ChunkedStream) { + // For now, assume the InputStream is doing proper chunking. + out.add(new EncryptedMessageWithHeader(in.body(), header, body, bodyLength)); + } else { + throw new IllegalArgumentException( + "Body must be a ByteBuf, ChunkedStream or an InputStream"); + } + } else { + out.add(header); + } + } +} diff --git a/common/src/main/java/org/apache/celeborn/common/network/sasl/CelebornSaslServer.java b/common/src/main/java/org/apache/celeborn/common/network/sasl/CelebornSaslServer.java index efb9af5a50..05c05744cd 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/sasl/CelebornSaslServer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/sasl/CelebornSaslServer.java @@ -37,6 +37,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.celeborn.common.network.client.TransportClient; + /** * A SASL Server for Celeborn which simply keeps track of the state of a single SASL session, from * the initial state to the "authenticated" state. (It is not a server in the sense of accepting @@ -106,6 +108,7 @@ public synchronized void dispose() { */ static class DigestCallbackHandler implements CallbackHandler { private final SecretRegistry secretRegistry; + private final TransportClient client; /** * The use of 'volatile' is not necessary here because the 'handle' invocation includes both the @@ -114,7 +117,8 @@ static class DigestCallbackHandler implements CallbackHandler { */ private String userName = null; - DigestCallbackHandler(SecretRegistry secretRegistry) { + DigestCallbackHandler(TransportClient client, SecretRegistry secretRegistry) { + this.client = Preconditions.checkNotNull(client); this.secretRegistry = Preconditions.checkNotNull(secretRegistry); } @@ -129,6 +133,7 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException, Sa throw new SaslException("No username provided by client"); } userName = decodeIdentifier(encodedName); + client.setClientId(userName); } else if (callback instanceof PasswordCallback) { logger.trace("SASL server callback: setting password"); PasswordCallback pc = (PasswordCallback) callback; diff --git a/common/src/main/java/org/apache/celeborn/common/network/sasl/SaslRpcHandler.java b/common/src/main/java/org/apache/celeborn/common/network/sasl/SaslRpcHandler.java index 127b3e7720..0033372eac 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/sasl/SaslRpcHandler.java +++ b/common/src/main/java/org/apache/celeborn/common/network/sasl/SaslRpcHandler.java @@ -94,7 +94,7 @@ public boolean doAuthChallenge( new CelebornSaslServer( DIGEST_MD5, DEFAULT_SASL_SERVER_PROPS, - new CelebornSaslServer.DigestCallbackHandler(secretRegistry)); + new CelebornSaslServer.DigestCallbackHandler(client, secretRegistry)); } byte[] response = saslServer.response(saslMessage.getPayload().toByteArray()); callback.onSuccess(ByteBuffer.wrap(response)); diff --git a/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationClientBootstrap.java b/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationClientBootstrap.java index 3604e927c7..a6aa2ba88f 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationClientBootstrap.java +++ b/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationClientBootstrap.java @@ -112,6 +112,7 @@ public void doBootstrap(TransportClient client) throws RuntimeException { register(client); LOG.info("Registration for {}", appId); registrationInfo.setRegistrationState(RegistrationInfo.RegistrationState.REGISTERED); + client.setClientId(appId); } catch (IOException | CelebornException e) { throw new RuntimeException(e); } finally { diff --git a/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationRpcHandler.java b/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationRpcHandler.java index 10803f95d9..c0b25e6f9a 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationRpcHandler.java +++ b/common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationRpcHandler.java @@ -181,6 +181,7 @@ private void processRpcMessage( LOG.trace("Application registration started {}", registerApplicationRequest.getId()); processRegisterApplicationRequest(registerApplicationRequest, callback); registrationState = RegistrationState.REGISTERED; + client.setClientId(registerApplicationRequest.getId()); LOG.info( "Application registered: appId {} rpcId {}", registerApplicationRequest.getId(), diff --git a/common/src/main/java/org/apache/celeborn/common/network/server/BaseMessageHandler.java b/common/src/main/java/org/apache/celeborn/common/network/server/BaseMessageHandler.java index d975dc4827..9741667455 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/server/BaseMessageHandler.java +++ b/common/src/main/java/org/apache/celeborn/common/network/server/BaseMessageHandler.java @@ -46,4 +46,12 @@ public void channelActive(TransportClient client) {} public void channelInactive(TransportClient client) {} public void exceptionCaught(Throwable cause, TransportClient client) {} + + protected void checkAuth(TransportClient client, String appId) { + if (client.getClientId() != null && !client.getClientId().equals(appId)) { + throw new SecurityException( + String.format( + "Client for %s not authorized for application %s.", client.getClientId(), appId)); + } + } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManager.java b/common/src/main/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManager.java new file mode 100644 index 0000000000..82f69f4f79 --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManager.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.ssl; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.util.concurrent.atomic.AtomicReference; + +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509TrustManager; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link TrustManager} implementation that reloads its configuration when the truststore file on + * disk changes. This implementation is based off of the + * org.apache.hadoop.security.ssl.ReloadingX509TrustManager class in the Apache Hadoop Encrypted + * Shuffle implementation. + * + * @see Hadoop + * MapReduce Next Generation - Encrypted Shuffle + */ +public class ReloadingX509TrustManager implements X509TrustManager, Runnable { + + private static final Logger logger = LoggerFactory.getLogger(ReloadingX509TrustManager.class); + + private final String type; + private final File file; + // The file being pointed to by `file` if it's a link + private String canonicalPath; + private final String password; + private long lastLoaded; + private final long reloadInterval; + @VisibleForTesting protected volatile int reloadCount; + @VisibleForTesting protected volatile int needsReloadCheckCounts; + private final AtomicReference trustManagerRef; + + private Thread reloader; + + /** + * Creates a reloadable trustmanager. The trustmanager reloads itself if the underlying trustore + * file has changed. + * + * @param type type of truststore file, typically 'jks'. + * @param trustStore the truststore file. + * @param password password of the truststore file. + * @param reloadInterval interval to check if the truststore file has changed, in milliseconds. + * @throws IOException thrown if the truststore could not be initialized due to an IO error. + * @throws GeneralSecurityException thrown if the truststore could not be initialized due to a + * security error. + */ + public ReloadingX509TrustManager( + String type, File trustStore, String password, long reloadInterval) + throws IOException, GeneralSecurityException { + this.type = type; + this.file = trustStore; + this.canonicalPath = this.file.getCanonicalPath(); + this.password = password; + this.trustManagerRef = new AtomicReference(); + this.trustManagerRef.set(loadTrustManager()); + this.reloadInterval = reloadInterval; + this.reloadCount = 0; + this.needsReloadCheckCounts = 0; + } + + /** Starts the reloader thread. */ + public void init() { + reloader = new Thread(this, "Truststore reloader thread"); + reloader.setDaemon(true); + reloader.start(); + } + + /** Stops the reloader thread. */ + public void destroy() throws InterruptedException { + reloader.interrupt(); + reloader.join(); + } + + /** + * Returns the reload check interval. + * + * @return the reload check interval, in milliseconds. + */ + public long getReloadInterval() { + return reloadInterval; + } + + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + X509TrustManager tm = trustManagerRef.get(); + if (tm != null) { + tm.checkClientTrusted(chain, authType); + } else { + throw new CertificateException( + "Unknown client chain certificate: " + + chain[0].toString() + + ". Please ensure the correct trust store is specified in the config"); + } + } + + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + X509TrustManager tm = trustManagerRef.get(); + if (tm != null) { + tm.checkServerTrusted(chain, authType); + } else { + throw new CertificateException( + "Unknown server chain certificate: " + + chain[0].toString() + + ". Please ensure the correct trust store is specified in the config"); + } + } + + private static final X509Certificate[] EMPTY = new X509Certificate[0]; + + @Override + public X509Certificate[] getAcceptedIssuers() { + X509Certificate[] issuers = EMPTY; + X509TrustManager tm = trustManagerRef.get(); + if (tm != null) { + issuers = tm.getAcceptedIssuers(); + } + return issuers; + } + + boolean needsReload() throws IOException { + boolean reload = true; + File latestCanonicalFile = file.getCanonicalFile(); + if (file.exists() && latestCanonicalFile.exists()) { + // `file` can be a symbolic link. We need to reload if it points to another file, + // or if the file has been modified + if (latestCanonicalFile.getPath().equals(canonicalPath) + && latestCanonicalFile.lastModified() == lastLoaded) { + reload = false; + } + } else { + lastLoaded = 0; + } + return reload; + } + + X509TrustManager loadTrustManager() throws IOException, GeneralSecurityException { + X509TrustManager trustManager = null; + KeyStore ks = KeyStore.getInstance(type); + File latestCanonicalFile = file.getCanonicalFile(); + canonicalPath = latestCanonicalFile.getPath(); + lastLoaded = latestCanonicalFile.lastModified(); + try (FileInputStream in = new FileInputStream(latestCanonicalFile)) { + char[] passwordCharacters = password != null ? password.toCharArray() : null; + ks.load(in, passwordCharacters); + logger.debug("Loaded truststore '" + file + "'"); + } + + TrustManagerFactory trustManagerFactory = + TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + trustManagerFactory.init(ks); + TrustManager[] trustManagers = trustManagerFactory.getTrustManagers(); + for (TrustManager trustManager1 : trustManagers) { + if (trustManager1 instanceof X509TrustManager) { + trustManager = (X509TrustManager) trustManager1; + break; + } + } + return trustManager; + } + + @Override + public void run() { + boolean running = true; + while (running) { + try { + Thread.sleep(reloadInterval); + } catch (InterruptedException e) { + running = false; + } + try { + if (running && needsReload()) { + try { + trustManagerRef.set(loadTrustManager()); + this.reloadCount += 1; + } catch (Exception ex) { + logger.warn( + "Could not load truststore (keep using existing one) : " + ex.toString(), ex); + } + } + } catch (IOException ex) { + logger.warn("Could not check whether truststore needs reloading: " + ex.toString(), ex); + } + needsReloadCheckCounts++; + } + } +} diff --git a/common/src/main/java/org/apache/celeborn/common/network/util/NettyLogger.java b/common/src/main/java/org/apache/celeborn/common/network/util/NettyLogger.java new file mode 100644 index 0000000000..eb7c5ffbfd --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/util/NettyLogger.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.util; + +import java.io.IOException; +import java.io.InputStream; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufHolder; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Netty logger that constructs a log handler depending on the log level. + * + *

Note: code copied from Apache Spark. + */ +public class NettyLogger { + private static final Logger logger = LoggerFactory.getLogger(NettyLogger.class); + + /** A Netty LoggingHandler which does not dump the message contents. */ + private static class NoContentLoggingHandler extends LoggingHandler { + + NoContentLoggingHandler(Class clazz, LogLevel level) { + super(clazz, level); + } + + @Override + protected String format(ChannelHandlerContext ctx, String eventName, Object arg) { + if (arg instanceof ByteBuf) { + return format(ctx, eventName) + " " + ((ByteBuf) arg).readableBytes() + "B"; + } else if (arg instanceof ByteBufHolder) { + return format(ctx, eventName) + " " + ((ByteBufHolder) arg).content().readableBytes() + "B"; + } else if (arg instanceof InputStream) { + int available = -1; + try { + available = ((InputStream) arg).available(); + } catch (IOException ex) { + // Swallow, but return -1 to indicate an error happened + } + return format(ctx, eventName, arg) + " " + available + "B"; + } else { + return super.format(ctx, eventName, arg); + } + } + } + + private final LoggingHandler loggingHandler; + + public NettyLogger() { + if (logger.isTraceEnabled()) { + loggingHandler = new LoggingHandler(NettyLogger.class, LogLevel.TRACE); + } else if (logger.isDebugEnabled()) { + loggingHandler = new NoContentLoggingHandler(NettyLogger.class, LogLevel.DEBUG); + } else { + loggingHandler = null; + } + } + + public LoggingHandler getLoggingHandler() { + return loggingHandler; + } +} diff --git a/common/src/main/java/org/apache/celeborn/common/network/util/TransportConf.java b/common/src/main/java/org/apache/celeborn/common/network/util/TransportConf.java index 417e933199..a1286b2852 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/util/TransportConf.java +++ b/common/src/main/java/org/apache/celeborn/common/network/util/TransportConf.java @@ -17,6 +17,8 @@ package org.apache.celeborn.common.network.util; +import java.io.File; + import org.apache.celeborn.common.CelebornConf; /** A central location that tracks all the settings we expose to users. */ @@ -163,4 +165,75 @@ public int saslTimeoutMs() { public boolean authEnabled() { return celebornConf.authEnabled(); } + + /** Whether Secure (SSL/TLS) wire communication is enabled. */ + public boolean sslEnabled() { + return celebornConf.sslEnabled(module); + } + + /** SSL protocol (remember that SSLv3 was compromised) supported by Java */ + public String sslProtocol() { + return celebornConf.sslProtocol(module); + } + + /** A comma separated list of ciphers */ + public String[] sslRequestedCiphers() { + return celebornConf.sslRequestedCiphers(module); + } + + /** The key-store file; can be relative to the current directory */ + public File sslKeyStore() { + return celebornConf.sslKeyStore(module); + } + + /** The password to the key-store file */ + public String sslKeyStorePassword() { + return celebornConf.sslKeyStorePassword(module); + } + + /** The trust-store file; can be relative to the current directory */ + public File sslTrustStore() { + return celebornConf.sslTrustStore(module); + } + + /** The password to the trust-store file */ + public String sslTrustStorePassword() { + return celebornConf.sslTrustStorePassword(module); + } + + /** + * If using a trust-store that that reloads its configuration is enabled. If true, when the + * trust-store file on disk changes, it will be reloaded + */ + public boolean sslTrustStoreReloadingEnabled() { + return celebornConf.sslTrustStoreReloadingEnabled(module); + } + + /** The interval, in milliseconds, the trust-store will reload its configuration */ + public int sslTrustStoreReloadIntervalMs() { + return celebornConf.sslTrustStoreReloadIntervalMs(module); + } + + /** Internal config: the max size when chunking the stream with SSL */ + public int maxSslEncryptedBlockSize() { + return celebornConf.maxSslEncryptedBlockSize(module); + } + + // suppressing to ensure clarity of code. + @SuppressWarnings("RedundantIfStatement") + public boolean sslEnabledAndKeysAreValid() { + if (!sslEnabled()) { + return false; + } + // It is not required to have a keyStore for client side connections - only server side + // connectivity ... so transport conf's without keystore can be used in + // client mode only. + // In case it is specified, we check for its validity + File keyStore = sslKeyStore(); + if (keyStore != null && !keyStore.exists()) { + return false; + } + // It's fine for the trust store to be missing, we would default to trusting all. + return true; + } } 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 9abd31bc2a..d5c26f45cd 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -17,7 +17,7 @@ package org.apache.celeborn.common -import java.io.IOException +import java.io.{File, IOException} import java.util.{Collection => JCollection, Collections, HashMap => JHashMap, Locale, Map => JMap} import java.util.concurrent.TimeUnit @@ -586,6 +586,10 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def masterHttpPort: Int = get(MASTER_HTTP_PORT) + def masterHttpMaxWorkerThreads: Int = get(MASTER_HTTP_MAX_WORKER_THREADS) + + def masterHttpStopTimeout: Long = get(MASTER_HTTP_STOP_TIMEOUT) + def haEnabled: Boolean = get(HA_ENABLED) def haMasterNodeId: Option[String] = get(HA_MASTER_NODE_ID) @@ -676,6 +680,8 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def workerHttpHost: String = get(WORKER_HTTP_HOST).replace("", Utils.localHostName(this)) def workerHttpPort: Int = get(WORKER_HTTP_PORT) + def workerHttpMaxWorkerThreads: Int = get(WORKER_HTTP_MAX_WORKER_THREADS) + def workerHttpStopTimeout: Long = get(WORKER_HTTP_STOP_TIMEOUT) def workerRpcPort: Int = get(WORKER_RPC_PORT) def workerPushPort: Int = get(WORKER_PUSH_PORT) def workerFetchPort: Int = get(WORKER_FETCH_PORT) @@ -1123,9 +1129,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def testPushReplicaDataTimeout: Boolean = get(TEST_WORKER_PUSH_REPLICA_DATA_TIMEOUT) def testRetryRevive: Boolean = get(TEST_CLIENT_RETRY_REVIVE) def testAlternative: String = get(TEST_ALTERNATIVE.key, "celeborn") - def clientFlinkMemoryPerResultPartitionMin: Long = get(CLIENT_MEMORY_PER_RESULT_PARTITION_MIN) def clientFlinkMemoryPerResultPartition: Long = get(CLIENT_MEMORY_PER_RESULT_PARTITION) - def clientFlinkMemoryPerInputGateMin: Long = get(CLIENT_MEMORY_PER_INPUT_GATE_MIN) def clientFlinkMemoryPerInputGate: Long = get(CLIENT_MEMORY_PER_INPUT_GATE) def clientFlinkNumConcurrentReading: Int = get(CLIENT_NUM_CONCURRENT_READINGS) def clientFlinkInputGateSupportFloatingBuffer: Boolean = @@ -1141,6 +1145,97 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def hdfsStorageKerberosPrincipal = get(HDFS_STORAGE_KERBEROS_PRINCIPAL) def hdfsStorageKerberosKeytab = get(HDFS_STORAGE_KERBEROS_KEYTAB) + // ////////////////////////////////////////////////////// + // TLS // + // ////////////////////////////////////////////////////// + private def getSslConfig[V](config: ConfigEntry[V], module: String): V = { + // For ssl, we look at the module specific value - and then fallback to without the + // module for global defaults, before falling back on what is in code + val moduleKey = config.key.replace("", module) + // replace the module wildcard and check for global value + val globalKey = config.key.replace("..", ".") + + val defaultValue = if (config.defaultValue.isDefined) config.defaultValueString else null + + config.valueConverter(getOption(moduleKey).getOrElse(get(globalKey, defaultValue))) + } + + private def asFileOrNull(fileName: Option[String]): File = { + fileName.map(new File(_)).orNull + } + + /** + * Whether Secure (SSL/TLS) wire communication is enabled. + */ + def sslEnabled(module: String): Boolean = { + getSslConfig(SSL_ENABLED, module) + } + + /** + * SSL protocol (remember that SSLv3 was compromised) supported by Java + */ + def sslProtocol(module: String): String = { + getSslConfig(SSL_PROTOCOL, module) + } + + /** + * A comma separated list of ciphers + */ + def sslRequestedCiphers(module: String): Array[String] = { + getSslConfig(SSL_ENABLED_CIPHERS, module).map(_.split(",")).orNull + } + + /** + * The key-store file; can be relative to the current directory + */ + def sslKeyStore(module: String): File = { + val keyStore = getSslConfig(SSL_KEY_STORE, module) + asFileOrNull(keyStore) + } + + /** + * The password to the key-store file + */ + def sslKeyStorePassword(module: String): String = { + getSslConfig(SSL_KEY_STORE_PASSWORD, module).orNull + } + + /** + * The trust-store file; can be relative to the current directory + */ + def sslTrustStore(module: String): File = { + asFileOrNull(getSslConfig(SSL_TRUST_STORE, module)) + } + + /** + * The password to the trust-store file + */ + def sslTrustStorePassword(module: String): String = { + getSslConfig(SSL_TRUST_STORE_PASSWORD, module).orNull + } + + /** + * If using a trust-store that that reloads its configuration is enabled. If true, + * when the trust-store file on disk changes, it will be reloaded + */ + def sslTrustStoreReloadingEnabled(module: String): Boolean = { + getSslConfig(SSL_TRUST_STORE_RELOADING_ENABLED, module) + } + + /** + * The interval, in milliseconds, the trust-store will reload its configuration + */ + def sslTrustStoreReloadIntervalMs(module: String): Int = { + getSslConfig(SSL_TRUST_STORE_RELOAD_INTERVAL_MS, module).toInt + } + + /** + * Internal config: the max size when chunking the stream with SSL + */ + def maxSslEncryptedBlockSize(module: String): Int = { + getSslConfig(MAX_SSL_ENCRYPTED_BLOCK_SIZE, module).toInt + } + // ////////////////////////////////////////////////////// // Authentication // // ////////////////////////////////////////////////////// @@ -1887,6 +1982,23 @@ object CelebornConf extends Logging { .checkValue(p => p >= 1024 && p < 65535, "Invalid port") .createWithDefault(9098) + val MASTER_HTTP_MAX_WORKER_THREADS: ConfigEntry[Int] = + buildConf("celeborn.master.http.maxWorkerThreads") + .categories("master") + .version("0.5.0") + .doc("Maximum number of threads in the master http worker thread pool.") + .intConf + .checkValue(_ > 0, "Must be positive.") + .createWithDefault(200) + + val MASTER_HTTP_STOP_TIMEOUT: ConfigEntry[Long] = + buildConf("celeborn.master.http.stopTimeout") + .categories("master") + .version("0.5.0") + .doc("Master http server stop timeout.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("5s") + val HA_ENABLED: ConfigEntry[Boolean] = buildConf("celeborn.master.ha.enabled") .withAlternative("celeborn.ha.enabled") @@ -2446,6 +2558,23 @@ object CelebornConf extends Logging { .checkValue(p => p >= 1024 && p < 65535, "Invalid port") .createWithDefault(9096) + val WORKER_HTTP_MAX_WORKER_THREADS: ConfigEntry[Int] = + buildConf("celeborn.worker.http.maxWorkerThreads") + .categories("worker") + .version("0.5.0") + .doc("Maximum number of threads in the worker http worker thread pool.") + .intConf + .checkValue(_ > 0, "Must be positive.") + .createWithDefault(200) + + val WORKER_HTTP_STOP_TIMEOUT: ConfigEntry[Long] = + buildConf("celeborn.worker.http.stopTimeout") + .categories("worker") + .version("0.5.0") + .doc("Worker http server stop timeout.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("5s") + val WORKER_RPC_PORT: ConfigEntry[Int] = buildConf("celeborn.worker.rpc.port") .categories("worker") @@ -4372,25 +4501,6 @@ object CelebornConf extends Logging { .booleanConf .createWithDefault(false) - // Flink specific client configurations. - val CLIENT_MEMORY_PER_RESULT_PARTITION_MIN: ConfigEntry[Long] = - buildConf("celeborn.client.flink.resultPartition.minMemory") - .withAlternative("remote-shuffle.job.min.memory-per-partition") - .categories("client") - .version("0.3.0") - .doc("Min memory reserved for a result partition.") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("8m") - - val CLIENT_MEMORY_PER_INPUT_GATE_MIN: ConfigEntry[Long] = - buildConf("celeborn.client.flink.inputGate.minMemory") - .withAlternative("remote-shuffle.job.min.memory-per-gate") - .categories("client") - .doc("Min memory reserved for a input gate.") - .version("0.3.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("8m") - val CLIENT_NUM_CONCURRENT_READINGS: ConfigEntry[Int] = buildConf("celeborn.client.flink.inputGate.concurrentReadings") .withAlternative("remote-shuffle.job.concurrent-readings-per-gate") @@ -4716,6 +4826,7 @@ object CelebornConf extends Logging { .doc("Number of threads used by the Master to send ApplicationMeta to Workers.") .version("0.5.0") .intConf + .checkValue(_ > 0, "number of threads should be positive") .createWithDefault(8) val WORKER_APPLICATION_REGISTRY_CACHE_SIZE: ConfigEntry[Int] = @@ -4725,4 +4836,105 @@ object CelebornConf extends Logging { .version("0.5.0") .intConf .createWithDefault(10000) + + // SSL Configs + + val SSL_ENABLED: ConfigEntry[Boolean] = + buildConf("celeborn.ssl..enabled") + .categories("network", "ssl") + .version("0.5.0") + .doc("Enables SSL for securing wire traffic.") + .booleanConf + .createWithDefault(false) + + val SSL_PROTOCOL: ConfigEntry[String] = + buildConf("celeborn.ssl..protocol") + .categories("network", "ssl") + .version("0.5.0") + .doc("SSL protocol to use") + .stringConf + // TLSv1.3 requires specific java version, defaulting to v1.2 + .createWithDefault("TLSv1.2") + + val SSL_ENABLED_CIPHERS: OptionalConfigEntry[String] = + buildConf("celeborn.ssl..enabledAlgorithms") + .categories("network", "ssl") + .version("0.5.0") + .doc("A comma-separated list of ciphers. The specified ciphers must be supported by JVM. " + + "The reference list of protocols can be found in the \"JSSE Cipher Suite Names\" section " + + "of the Java security guide. The list for Java 17 can be found at " + + "https://docs.oracle.com/en/java/javase/17/docs/specs/security/standard-names.html#jsse-cipher-suite-names " + + ". Note: If not set, the default cipher suite for the JRE will be used.") + .stringConf + .createOptional + + val SSL_KEY_STORE: OptionalConfigEntry[String] = + buildConf("celeborn.ssl..keyStore") + .categories("network", "ssl") + .version("0.5.0") + .doc("Path to the key store file. The path can be absolute or relative to the directory in which the " + + "process is started.") + .stringConf + .createOptional + + val SSL_KEY_STORE_PASSWORD: OptionalConfigEntry[String] = + buildConf("celeborn.ssl..keyStorePassword") + .categories("network", "ssl") + .version("0.5.0") + .doc("Password to the key store.") + .stringConf + .createOptional + + val SSL_TRUST_STORE: OptionalConfigEntry[String] = + buildConf("celeborn.ssl..trustStore") + .categories("network", "ssl") + .version("0.5.0") + .doc("Path to the trust store file. The path can be absolute or relative to the directory " + + "in which the process is started.") + .stringConf + .createOptional + + val SSL_TRUST_STORE_PASSWORD: OptionalConfigEntry[String] = + buildConf("celeborn.ssl..trustStorePassword") + .categories("network", "ssl") + .version("0.5.0") + .doc("Password for the trust store.") + .stringConf + .createOptional + + val SSL_TRUST_STORE_RELOADING_ENABLED: ConfigEntry[Boolean] = + buildConf("celeborn.ssl..trustStoreReloadingEnabled") + .categories("network", "ssl") + .version("0.5.0") + .doc("Whether the trust store should be reloaded periodically. This setting is mostly only " + + "useful for server components, not applications.") + .booleanConf + .createWithDefault(false) + + val SSL_TRUST_STORE_RELOAD_INTERVAL_MS: ConfigEntry[Long] = + buildConf("celeborn.ssl..trustStoreReloadIntervalMs") + .categories("network", "ssl") + .version("0.5.0") + .doc("The interval at which the trust store should be reloaded (in milliseconds). This " + + "setting is mostly only useful for server components, not applications.") + .timeConf(TimeUnit.MILLISECONDS) + // We treat this as an int, so validate + .checkValue( + p => p > 0 && p <= Int.MaxValue, + s"Invalid trustStoreReloadIntervalMs, must be a position number upto ${Int.MaxValue}") + .createWithDefaultString("10s") + + val MAX_SSL_ENCRYPTED_BLOCK_SIZE: ConfigEntry[Long] = + buildConf("celeborn.ssl..maxEncryptedBlockSize") + .categories("network", "ssl") + .version("0.5.0") + .internal + .doc("The max size when chunking the stream with SSL") + .bytesConf(ByteUnit.BYTE) + // We treat this as an int, so validate + .checkValue( + p => p > 0 && p <= Int.MaxValue, + s"Invalid maxEncryptedBlockSize, must be a position number upto ${Int.MaxValue}") + .createWithDefaultString("64k") + } diff --git a/common/src/main/scala/org/apache/celeborn/common/internal/config/ConfigEntry.scala b/common/src/main/scala/org/apache/celeborn/common/internal/config/ConfigEntry.scala index 344bd1aff6..bfd9cca769 100644 --- a/common/src/main/scala/org/apache/celeborn/common/internal/config/ConfigEntry.scala +++ b/common/src/main/scala/org/apache/celeborn/common/internal/config/ConfigEntry.scala @@ -240,7 +240,7 @@ class OptionalConfigEntry[T]( prependedKey, prependSeparator, alternatives, - s => Some(rawValueConverter(s)), + s => Option(rawValueConverter(s)), v => v.map(rawStringConverter).orNull, doc, isPublic, diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServerInitializer.scala b/common/src/main/scala/org/apache/celeborn/common/metrics/MetricsUtils.scala similarity index 57% rename from service/src/main/scala/org/apache/celeborn/server/common/http/HttpServerInitializer.scala rename to common/src/main/scala/org/apache/celeborn/common/metrics/MetricsUtils.scala index 2619606b5b..0d5086bbbf 100644 --- a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServerInitializer.scala +++ b/common/src/main/scala/org/apache/celeborn/common/metrics/MetricsUtils.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.celeborn.server.common.http +package org.apache.celeborn.common.metrics -import io.netty.channel.{ChannelInitializer, SimpleChannelInboundHandler} -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.http.{HttpObjectAggregator, HttpServerCodec} +import java.util.concurrent.TimeUnit -class HttpServerInitializer( - handlers: SimpleChannelInboundHandler[_]) extends ChannelInitializer[SocketChannel] { +object MetricsUtils { + private[this] val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + private[this] val MINIMAL_POLL_PERIOD = 1 - override def initChannel(channel: SocketChannel): Unit = { - val pipeline = channel.pipeline() - pipeline.addLast(new HttpServerCodec()) - .addLast("httpAggregator", new HttpObjectAggregator(512 * 1024)) - .addLast(handlers) + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } } } diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/CsvSink.scala b/common/src/main/scala/org/apache/celeborn/common/metrics/sink/CsvSink.scala index 34097b34d6..52ccf46389 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/CsvSink.scala +++ b/common/src/main/scala/org/apache/celeborn/common/metrics/sink/CsvSink.scala @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} -import org.apache.celeborn.common.metrics.MetricsSystem +import org.apache.celeborn.common.metrics.MetricsUtils class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val CSV_KEY_PERIOD = "period" @@ -44,7 +44,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } - MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + MetricsUtils.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/GraphiteSink.scala b/common/src/main/scala/org/apache/celeborn/common/metrics/sink/GraphiteSink.scala index cc97b9bef2..b2be72fff1 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/GraphiteSink.scala +++ b/common/src/main/scala/org/apache/celeborn/common/metrics/sink/GraphiteSink.scala @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry import com.codahale.metrics.graphite.{Graphite, GraphiteReporter, GraphiteUDP} -import org.apache.celeborn.common.metrics.MetricsSystem +import org.apache.celeborn.common.metrics.MetricsUtils private class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 @@ -62,7 +62,7 @@ private class GraphiteSink(val property: Properties, val registry: MetricRegistr val prefix = propertyToOption(GRAPHITE_KEY_PREFIX).getOrElse(GRAPHITE_DEFAULT_PREFIX) - MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + MetricsUtils.checkMinimalPollingPeriod(pollUnit, pollPeriod) val graphite = propertyToOption(GRAPHITE_KEY_PROTOCOL).map(_.toLowerCase(Locale.ROOT)) match { case Some("udp") => new GraphiteUDP(host, port) diff --git a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala index 82946dfb7b..9bd9469b07 100644 --- a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala +++ b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala @@ -891,6 +891,12 @@ object ControlMessages extends Logging { case pb: PbCheckWorkersAvailableResponse => new TransportMessage(MessageType.CHECK_WORKERS_AVAILABLE_RESPONSE, pb.toByteArray) + + case pb: PbApplicationMeta => + new TransportMessage(MessageType.APPLICATION_META, pb.toByteArray) + + case pb: PbApplicationMetaRequest => + new TransportMessage(MessageType.APPLICATION_META_REQUEST, pb.toByteArray) } // TODO change return type to GeneratedMessageV3 @@ -1238,6 +1244,12 @@ object ControlMessages extends Logging { case CHECK_WORKERS_AVAILABLE_RESPONSE_VALUE => PbCheckWorkersAvailableResponse.parseFrom(message.getPayload) + + case APPLICATION_META_VALUE => + PbApplicationMeta.parseFrom(message.getPayload) + + case APPLICATION_META_REQUEST_VALUE => + PbApplicationMetaRequest.parseFrom(message.getPayload) } } } diff --git a/common/src/test/java/org/apache/celeborn/common/network/TestHelper.java b/common/src/test/java/org/apache/celeborn/common/network/TestHelper.java new file mode 100644 index 0000000000..1e9c1cb48c --- /dev/null +++ b/common/src/test/java/org/apache/celeborn/common/network/TestHelper.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network; + +import java.util.Map; + +import org.apache.celeborn.common.CelebornConf; + +/** A few helper utilities to reduce duplication within test code. */ +public class TestHelper { + + public static CelebornConf updateCelebornConfWithMap(CelebornConf conf, Map map) { + for (Map.Entry entry : map.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + return conf; + } +} diff --git a/common/src/test/java/org/apache/celeborn/common/network/TestManagedBuffer.java b/common/src/test/java/org/apache/celeborn/common/network/TestManagedBuffer.java index b5f196fe2f..ad3cc45212 100644 --- a/common/src/test/java/org/apache/celeborn/common/network/TestManagedBuffer.java +++ b/common/src/test/java/org/apache/celeborn/common/network/TestManagedBuffer.java @@ -79,6 +79,11 @@ public Object convertToNetty() throws IOException { return underlying.convertToNetty(); } + @Override + public Object convertToNettyForSsl() throws IOException { + return underlying.convertToNettyForSsl(); + } + @Override public int hashCode() { return underlying.hashCode(); diff --git a/common/src/test/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeaderSuiteJ.java b/common/src/test/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeaderSuiteJ.java new file mode 100644 index 0000000000..0fbf7e9c9a --- /dev/null +++ b/common/src/test/java/org/apache/celeborn/common/network/protocol/EncryptedMessageWithHeaderSuiteJ.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.protocol; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Random; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; +import io.netty.handler.stream.ChunkedStream; +import org.junit.Test; + +import org.apache.celeborn.common.network.buffer.ManagedBuffer; +import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; + +/* + * Based on common/network-common/org.apache.spark.network.protocol.EncryptedMessageWithHeaderSuite + */ +public class EncryptedMessageWithHeaderSuiteJ { + + // Tests the case where the body is an input stream and that we manage the refcounts of the + // buffer properly + @Test + public void testInputStreamBodyFromManagedBuffer() throws Exception { + byte[] randomData = new byte[128]; + new Random().nextBytes(randomData); + ByteBuf sourceBuffer = Unpooled.copiedBuffer(randomData); + InputStream body = new ByteArrayInputStream(sourceBuffer.array()); + ByteBuf header = Unpooled.copyLong(42); + + long expectedHeaderValue = header.getLong(header.readerIndex()); + assertEquals(1, header.refCnt()); + assertEquals(1, sourceBuffer.refCnt()); + ManagedBuffer managedBuf = new NettyManagedBuffer(sourceBuffer); + + EncryptedMessageWithHeader msg = + new EncryptedMessageWithHeader(managedBuf, header, body, managedBuf.size()); + ByteBufAllocator allocator = ByteBufAllocator.DEFAULT; + + // First read should just read the header + ByteBuf headerResult = msg.readChunk(allocator); + assertEquals(header.capacity(), headerResult.readableBytes()); + assertEquals(expectedHeaderValue, headerResult.readLong()); + assertEquals(header.capacity(), msg.progress()); + assertFalse(msg.isEndOfInput()); + + // Second read should read the body + ByteBuf bodyResult = msg.readChunk(allocator); + assertEquals(randomData.length + header.capacity(), msg.progress()); + assertTrue(msg.isEndOfInput()); + + // Validate we read it all + assertEquals(bodyResult.readableBytes(), randomData.length); + for (int i = 0; i < randomData.length; i++) { + assertEquals(bodyResult.readByte(), randomData[i]); + } + + // Closing the message should release the source buffer + msg.close(); + assertEquals(0, sourceBuffer.refCnt()); + + // The header still has a reference we got + assertEquals(1, header.refCnt()); + headerResult.release(); + assertEquals(0, header.refCnt()); + } + + // Tests the case where the body is a chunked stream and that we are fine when there is no + // input managed buffer + @Test + public void testChunkedStream() throws Exception { + int bodyLength = 129; + int chunkSize = 64; + byte[] randomData = new byte[bodyLength]; + new Random().nextBytes(randomData); + InputStream inputStream = new ByteArrayInputStream(randomData); + ChunkedStream body = new ChunkedStream(inputStream, chunkSize); + ByteBuf header = Unpooled.copyLong(42); + + long expectedHeaderValue = header.getLong(header.readerIndex()); + assertEquals(1, header.refCnt()); + + EncryptedMessageWithHeader msg = new EncryptedMessageWithHeader(null, header, body, bodyLength); + ByteBufAllocator allocator = ByteBufAllocator.DEFAULT; + + // First read should just read the header + ByteBuf headerResult = msg.readChunk(allocator); + assertEquals(header.capacity(), headerResult.readableBytes()); + assertEquals(expectedHeaderValue, headerResult.readLong()); + assertEquals(header.capacity(), msg.progress()); + assertFalse(msg.isEndOfInput()); + + // Next 2 reads should read full buffers + int readIndex = 0; + for (int i = 1; i <= 2; i++) { + ByteBuf bodyResult = msg.readChunk(allocator); + assertEquals(header.capacity() + (i * chunkSize), msg.progress()); + assertFalse(msg.isEndOfInput()); + + // Validate we read data correctly + assertEquals(bodyResult.readableBytes(), chunkSize); + assert (bodyResult.readableBytes() < (randomData.length - readIndex)); + while (bodyResult.readableBytes() > 0) { + assertEquals(bodyResult.readByte(), randomData[readIndex++]); + } + } + + // Last read should be partial + ByteBuf bodyResult = msg.readChunk(allocator); + assertEquals(header.capacity() + bodyLength, msg.progress()); + assertTrue(msg.isEndOfInput()); + + // Validate we read the byte properly + assertEquals(bodyResult.readableBytes(), 1); + assertEquals(bodyResult.readByte(), randomData[readIndex]); + + // Closing the message should close the input stream + msg.close(); + assertTrue(body.isEndOfInput()); + + // The header still has a reference we got + assertEquals(1, header.refCnt()); + headerResult.release(); + assertEquals(0, header.refCnt()); + } + + @Test + public void testByteBufIsNotSupported() throws Exception { + // Validate that ByteBufs are not supported. This test can be updated + // when we add support for them + ByteBuf header = Unpooled.copyLong(42); + assertThrows( + IllegalArgumentException.class, + () -> { + EncryptedMessageWithHeader msg = new EncryptedMessageWithHeader(null, header, header, 4); + }); + } +} diff --git a/common/src/test/java/org/apache/celeborn/common/network/sasl/CelebornSaslSuiteJ.java b/common/src/test/java/org/apache/celeborn/common/network/sasl/CelebornSaslSuiteJ.java index b97da2e864..1a24a8ddc9 100644 --- a/common/src/test/java/org/apache/celeborn/common/network/sasl/CelebornSaslSuiteJ.java +++ b/common/src/test/java/org/apache/celeborn/common/network/sasl/CelebornSaslSuiteJ.java @@ -24,6 +24,7 @@ import org.junit.Test; import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.common.network.client.TransportClient; import org.apache.celeborn.common.network.server.BaseMessageHandler; import org.apache.celeborn.common.network.util.TransportConf; @@ -39,11 +40,12 @@ public void testDigestMatching() { DIGEST_MD5, DEFAULT_SASL_CLIENT_PROPS, new CelebornSaslClient.ClientCallbackHandler(TEST_USER, TEST_SECRET)); + TransportClient transportClient = mock(TransportClient.class); CelebornSaslServer server = new CelebornSaslServer( DIGEST_MD5, DEFAULT_SASL_SERVER_PROPS, - new CelebornSaslServer.DigestCallbackHandler(secretRegistry)); + new CelebornSaslServer.DigestCallbackHandler(transportClient, secretRegistry)); assertFalse(client.isComplete()); assertFalse(server.isComplete()); @@ -54,6 +56,7 @@ public void testDigestMatching() { clientMessage = client.response(server.response(clientMessage)); } assertTrue(server.isComplete()); + verify(transportClient, times(1)).setClientId(TEST_USER); // Disposal should invalidate server.dispose(); @@ -73,7 +76,8 @@ public void testDigestNonMatching() { new CelebornSaslServer( DIGEST_MD5, DEFAULT_SASL_SERVER_PROPS, - new CelebornSaslServer.DigestCallbackHandler(secretRegistry)); + new CelebornSaslServer.DigestCallbackHandler( + mock(TransportClient.class), secretRegistry)); assertFalse(client.isComplete()); assertFalse(server.isComplete()); diff --git a/common/src/test/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManagerSuiteJ.java b/common/src/test/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManagerSuiteJ.java new file mode 100644 index 0000000000..9b79662d33 --- /dev/null +++ b/common/src/test/java/org/apache/celeborn/common/network/ssl/ReloadingX509TrustManagerSuiteJ.java @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.celeborn.common.network.ssl; + +import static org.apache.celeborn.common.network.ssl.SslSampleConfigs.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.security.KeyPair; +import java.security.KeyStore; +import java.security.cert.X509Certificate; +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +public class ReloadingX509TrustManagerSuiteJ { + + /** + * Waits until reload count hits the requested value, sleeping 100ms at a time. If the maximum + * number of attempts is hit, throws a RuntimeException + * + * @param tm the trust manager to wait for + * @param count The count to wait for + * @param attempts The number of attempts to wait for + */ + private void waitForReloadCount(ReloadingX509TrustManager tm, int count, int attempts) + throws InterruptedException { + if (tm.reloadCount > count) { + throw new IllegalStateException( + "Passed invalid count " + + count + + " to waitForReloadCount, already have " + + tm.reloadCount); + } + for (int i = 0; i < attempts; i++) { + if (tm.reloadCount >= count) { + return; + } + // Adapted from SystemClock.waitTillTime + long startTime = System.currentTimeMillis(); + long targetTime = startTime + 100; + long currentTime = startTime; + while (currentTime < targetTime) { + long sleepTime = Math.min(10, targetTime - currentTime); + Thread.sleep(sleepTime); + currentTime = System.currentTimeMillis(); + } + } + throw new IllegalStateException("Trust store not reloaded after " + attempts + " attempts!"); + } + + /** + * Waits until we make some number of attempts to reload, and verifies that the actual reload + * count did not change + * + * @param tm the trust manager to wait for + * @param attempts The number of attempts to wait for + */ + private void waitForNoReload(ReloadingX509TrustManager tm, int attempts) + throws InterruptedException { + int oldReloadCount = tm.reloadCount; + int checkCount = tm.needsReloadCheckCounts; + int target = checkCount + attempts; + while (checkCount < target) { + Thread.sleep(100); + checkCount = tm.needsReloadCheckCounts; + } + assertEquals(oldReloadCount, tm.reloadCount); + } + + /** + * Tests to ensure that loading a missing trust-store fails + * + * @throws Exception + */ + @Test + public void testLoadMissingTrustStore() throws Exception { + File trustStore = new File("testmissing.jks"); + assertFalse(trustStore.exists()); + + assertThrows( + IOException.class, + () -> { + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager(KeyStore.getDefaultType(), trustStore, "password", 10); + try { + tm.init(); + } finally { + tm.destroy(); + } + }); + } + + /** + * Tests to ensure that loading a corrupt trust-store fails + * + * @throws Exception + */ + @Test + public void testLoadCorruptTrustStore() throws Exception { + File corruptStore = File.createTempFile("truststore-corrupt", "jks"); + corruptStore.deleteOnExit(); + OutputStream os = new FileOutputStream(corruptStore); + os.write(1); + os.close(); + + assertThrows( + IOException.class, + () -> { + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager( + KeyStore.getDefaultType(), corruptStore, "password", 10); + try { + tm.init(); + } finally { + tm.destroy(); + corruptStore.delete(); + } + }); + } + + /** + * Tests that we successfully reload when a file is updated + * + * @throws Exception + */ + @Test + public void testReload() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + X509Certificate cert2 = generateCertificate("CN=Cert2", kp, 30, "SHA1withRSA"); + File trustStore = File.createTempFile("testreload", "jks"); + trustStore.deleteOnExit(); + createTrustStore(trustStore, "password", "cert1", cert1); + + ReloadingX509TrustManager tm = new ReloadingX509TrustManager("jks", trustStore, "password", 1); + assertEquals(1, tm.getReloadInterval()); + assertEquals(0, tm.reloadCount); + try { + tm.init(); + assertEquals(1, tm.getAcceptedIssuers().length); + // At this point we haven't reloaded, just the initial load + assertEquals(0, tm.reloadCount); + + // Wait so that the file modification time is different + Thread.sleep((tm.getReloadInterval() + 1000)); + + // Add another cert + Map certs = new HashMap(); + certs.put("cert1", cert1); + certs.put("cert2", cert2); + createTrustStore(trustStore, "password", certs); + + // Wait up to 10s until we reload + waitForReloadCount(tm, 1, 100); + + assertEquals(2, tm.getAcceptedIssuers().length); + } finally { + tm.destroy(); + trustStore.delete(); + } + } + + /** + * Tests that we keep old certs if the trust store goes missing + * + * @throws Exception + */ + @Test + public void testReloadMissingTrustStore() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + File trustStore = new File("testmissing.jks"); + trustStore.deleteOnExit(); + assertFalse(trustStore.exists()); + createTrustStore(trustStore, "password", "cert1", cert1); + + ReloadingX509TrustManager tm = new ReloadingX509TrustManager("jks", trustStore, "password", 1); + assertEquals(0, tm.reloadCount); + try { + tm.init(); + assertEquals(1, tm.getAcceptedIssuers().length); + X509Certificate cert = tm.getAcceptedIssuers()[0]; + trustStore.delete(); + + // Wait for up to 5s - we should *not* reload + waitForNoReload(tm, 50); + + assertEquals(1, tm.getAcceptedIssuers().length); + assertEquals(cert, tm.getAcceptedIssuers()[0]); + } finally { + tm.destroy(); + } + } + + /** + * Tests that we keep old certs if the new truststore is corrupt + * + * @throws Exception + */ + @Test + public void testReloadCorruptTrustStore() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + File corruptStore = File.createTempFile("truststore-corrupt", "jks"); + corruptStore.deleteOnExit(); + createTrustStore(corruptStore, "password", "cert1", cert1); + + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager("jks", corruptStore, "password", 1); + assertEquals(0, tm.reloadCount); + try { + tm.init(); + assertEquals(1, tm.getAcceptedIssuers().length); + X509Certificate cert = tm.getAcceptedIssuers()[0]; + + OutputStream os = new FileOutputStream(corruptStore); + os.write(1); + os.close(); + corruptStore.setLastModified(System.currentTimeMillis() - 1000); + + // Wait for up to 5s - we should *not* reload + waitForNoReload(tm, 50); + + assertEquals(1, tm.getAcceptedIssuers().length); + assertEquals(cert, tm.getAcceptedIssuers()[0]); + } finally { + tm.destroy(); + corruptStore.delete(); + } + } + + /** + * Tests that we successfully reload when the trust store is a symlink and we update the contents + * of the pointed-to file or we update the file it points to. + * + * @throws Exception + */ + @Test + public void testReloadSymlink() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + X509Certificate cert2 = generateCertificate("CN=Cert2", kp, 30, "SHA1withRSA"); + X509Certificate cert3 = generateCertificate("CN=Cert3", kp, 30, "SHA1withRSA"); + + File trustStore1 = File.createTempFile("testreload", "jks"); + trustStore1.deleteOnExit(); + createTrustStore(trustStore1, "password", "cert1", cert1); + + File trustStore2 = File.createTempFile("testreload", "jks"); + Map certs = new HashMap(); + certs.put("cert1", cert1); + certs.put("cert2", cert2); + createTrustStore(trustStore2, "password", certs); + + File trustStoreSymlink = File.createTempFile("testreloadsymlink", "jks"); + trustStoreSymlink.delete(); + Files.createSymbolicLink(trustStoreSymlink.toPath(), trustStore1.toPath()); + + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager("jks", trustStoreSymlink, "password", 1); + assertEquals(1, tm.getReloadInterval()); + assertEquals(0, tm.reloadCount); + try { + tm.init(); + assertEquals(1, tm.getAcceptedIssuers().length); + // At this point we haven't reloaded, just the initial load + assertEquals(0, tm.reloadCount); + + // Repoint to trustStore2, which has another cert + trustStoreSymlink.delete(); + Files.createSymbolicLink(trustStoreSymlink.toPath(), trustStore2.toPath()); + + // Wait up to 10s until we reload + waitForReloadCount(tm, 1, 100); + + assertEquals(2, tm.getAcceptedIssuers().length); + + // Add another cert + certs.put("cert3", cert3); + createTrustStore(trustStore2, "password", certs); + + // Wait up to 10s until we reload + waitForReloadCount(tm, 2, 100); + + assertEquals(3, tm.getAcceptedIssuers().length); + } finally { + tm.destroy(); + trustStore1.delete(); + trustStore2.delete(); + trustStoreSymlink.delete(); + } + } +} diff --git a/common/src/test/java/org/apache/celeborn/common/network/ssl/SslSampleConfigs.java b/common/src/test/java/org/apache/celeborn/common/network/ssl/SslSampleConfigs.java new file mode 100644 index 0000000000..653709a31c --- /dev/null +++ b/common/src/test/java/org/apache/celeborn/common/network/ssl/SslSampleConfigs.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.ssl; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.math.BigInteger; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; +import java.security.*; +import java.security.cert.Certificate; +import java.security.cert.CertificateEncodingException; +import java.security.cert.X509Certificate; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; + +import javax.security.auth.x500.X500Principal; + +import org.apache.commons.io.FileUtils; +import org.bouncycastle.x509.X509V1CertificateGenerator; + +public class SslSampleConfigs { + + public static final String DEFAULT_KEY_STORE_PATH = getResourceAsAbsolutePath("/ssl/server.jks"); + public static final String SECOND_KEY_STORE_PATH = + getResourceAsAbsolutePath("/ssl/server_another.jks"); + + // trust store has ca's for both keys. + public static final String TRUST_STORE_PATH = getResourceAsAbsolutePath("/ssl/truststore.jks"); + + // this is a trust store which does not have either the primary or second cert's ca + public static final String TRUST_STORE_WITHOUT_CA = + getResourceAsAbsolutePath("/ssl/truststore-without-ca.jks"); + + public static Map createDefaultConfigMapForModule(String module) { + return createConfigMapForModule(module, true); + } + + public static Map createAnotherConfigMapForModule(String module) { + return createConfigMapForModule(module, false); + } + + private static Map createConfigMapForModule(String module, boolean forDefault) { + Map confMap = new HashMap<>(); + confMap.put("celeborn.ssl." + module + ".enabled", "true"); + confMap.put("celeborn.ssl." + module + ".trustStoreReloadingEnabled", "false"); + confMap.put("celeborn.ssl." + module + ".openSslEnabled", "false"); + confMap.put("celeborn.ssl." + module + ".trustStoreReloadIntervalMs", "10000"); + if (forDefault) { + confMap.put("celeborn.ssl." + module + ".keyStore", DEFAULT_KEY_STORE_PATH); + } else { + confMap.put("celeborn.ssl." + module + ".keyStore", SECOND_KEY_STORE_PATH); + } + confMap.put("celeborn.ssl." + module + ".keyStorePassword", "password"); + confMap.put("celeborn.ssl." + module + ".keyPassword", "password"); + confMap.put("celeborn.ssl." + module + ".privateKeyPassword", "password"); + confMap.put("celeborn.ssl." + module + ".protocol", "TLSv1.2"); + confMap.put("celeborn.ssl." + module + ".trustStore", TRUST_STORE_PATH); + confMap.put("celeborn.ssl." + module + ".trustStorePassword", "password"); + return confMap; + } + + public static void createTrustStore( + File trustStore, String password, String alias, Certificate cert) + throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setCertificateEntry(alias, cert); + saveKeyStore(ks, trustStore, password); + } + + /** Creates a keystore with multiple keys and saves it to a file. */ + public static void createTrustStore( + File trustStore, String password, Map certs) + throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + for (Map.Entry cert : certs.entrySet()) { + ks.setCertificateEntry(cert.getKey(), cert.getValue()); + } + saveKeyStore(ks, trustStore, password); + } + + /** + * Create a self-signed X.509 Certificate. + * + * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB" + * @param pair the KeyPair + * @param days how many days from now the Certificate is valid for + * @param algorithm the signing algorithm, eg "SHA1withRSA" + * @return the self-signed certificate + */ + @SuppressWarnings("deprecation") + public static X509Certificate generateCertificate( + String dn, KeyPair pair, int days, String algorithm) + throws CertificateEncodingException, InvalidKeyException, IllegalStateException, + NoSuchAlgorithmException, SignatureException { + + Date from = new Date(); + Date to = new Date(from.getTime() + days * 86400000L); + BigInteger sn = new BigInteger(64, new SecureRandom()); + KeyPair keyPair = pair; + X509V1CertificateGenerator certGen = new X509V1CertificateGenerator(); + X500Principal dnName = new X500Principal(dn); + + certGen.setSerialNumber(sn); + certGen.setIssuerDN(dnName); + certGen.setNotBefore(from); + certGen.setNotAfter(to); + certGen.setSubjectDN(dnName); + certGen.setPublicKey(keyPair.getPublic()); + certGen.setSignatureAlgorithm(algorithm); + + X509Certificate cert = certGen.generate(pair.getPrivate()); + return cert; + } + + public static KeyPair generateKeyPair(String algorithm) throws NoSuchAlgorithmException { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm); + keyGen.initialize(1024); + return keyGen.genKeyPair(); + } + + /** + * Creates a keystore with a single key and saves it to a file. + * + * @param keyStore File keystore to save + * @param password String store password to set on keystore + * @param keyPassword String key password to set on key + * @param alias String alias to use for the key + * @param privateKey Key to save in keystore + * @param cert Certificate to use as certificate chain associated to key + * @throws GeneralSecurityException for any error with the security APIs + * @throws IOException if there is an I/O error saving the file + */ + public static void createKeyStore( + File keyStore, + String password, + String keyPassword, + String alias, + Key privateKey, + Certificate cert) + throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, keyPassword.toCharArray(), new Certificate[] {cert}); + saveKeyStore(ks, keyStore, password); + } + + public static void createKeyStore( + File keyStore, String password, String alias, Key privateKey, Certificate cert) + throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, password.toCharArray(), new Certificate[] {cert}); + saveKeyStore(ks, keyStore, password); + } + + private static KeyStore createEmptyKeyStore() throws GeneralSecurityException, IOException { + KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType()); + ks.load(null, null); // initialize + return ks; + } + + private static void saveKeyStore(KeyStore ks, File keyStore, String password) + throws GeneralSecurityException, IOException { + // Write the file atomically to ensure tests don't read a partial write + File tempFile = File.createTempFile("temp-key-store", "jks"); + FileOutputStream out = new FileOutputStream(tempFile); + try { + ks.store(out, password.toCharArray()); + out.close(); + Files.move( + tempFile.toPath(), + keyStore.toPath(), + StandardCopyOption.REPLACE_EXISTING, + StandardCopyOption.ATOMIC_MOVE); + } finally { + out.close(); + } + } + + public static String getResourceAsAbsolutePath(String path) { + try { + File tempFile = File.createTempFile(new File(path).getName(), null); + tempFile.deleteOnExit(); + URL url = SslSampleConfigs.class.getResource(path); + if (null == url) { + throw new IllegalArgumentException("Unable to find " + path); + } + FileUtils.copyInputStreamToFile(url.openStream(), tempFile); + return tempFile.getCanonicalPath(); + } catch (IOException e) { + throw new RuntimeException("Failed to resolve path " + path, e); + } + } +} diff --git a/common/src/test/java/org/apache/celeborn/common/network/util/TransportConfSuiteJ.java b/common/src/test/java/org/apache/celeborn/common/network/util/TransportConfSuiteJ.java new file mode 100644 index 0000000000..92288b21a2 --- /dev/null +++ b/common/src/test/java/org/apache/celeborn/common/network/util/TransportConfSuiteJ.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.common.network.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; + +import org.junit.Test; + +import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.common.network.TestHelper; +import org.apache.celeborn.common.network.ssl.SslSampleConfigs; + +public class TransportConfSuiteJ { + + private TransportConf transportConf = + new TransportConf( + "rpc", + TestHelper.updateCelebornConfWithMap( + new CelebornConf(), SslSampleConfigs.createDefaultConfigMapForModule("rpc"))); + + @Test + public void testKeyStorePath() { + assertEquals(new File(SslSampleConfigs.DEFAULT_KEY_STORE_PATH), transportConf.sslKeyStore()); + } + + @Test + public void testTrustStorePath() { + assertEquals(new File(SslSampleConfigs.TRUST_STORE_PATH), transportConf.sslTrustStore()); + } + + @Test + public void testTrustStoreReloadingEnabled() { + assertFalse(transportConf.sslTrustStoreReloadingEnabled()); + } + + @Test + public void testSslEnabled() { + assertTrue(transportConf.sslEnabled()); + } + + @Test + public void testSslKeyStorePassword() { + assertEquals("password", transportConf.sslKeyStorePassword()); + } + + @Test + public void testSslTrustStorePassword() { + assertEquals("password", transportConf.sslTrustStorePassword()); + } + + @Test + public void testSsltrustStoreReloadIntervalMs() { + assertEquals(10000, transportConf.sslTrustStoreReloadIntervalMs()); + } + + // If a specific key is not set, it should be inherited from celeborn.ssl namespace + @Test + public void testInheritance() { + + final String module1 = "rpc"; + final String module2 = "fetch"; + + final String module1Protocol = "456"; + final String module2Protocol = "789"; + + final long module1ReloadIntervalMs = 123456; + final long defaultReloadIntervalMs = 83723; + + CelebornConf conf = new CelebornConf(); + + // Both should be independently working + conf.set("celeborn.ssl." + module1 + ".protocol", module1Protocol); + conf.set("celeborn.ssl." + module2 + ".protocol", module2Protocol); + + // setting at celeborn.ssl should inherit for module2 as it is not overriden + conf.set( + "celeborn.ssl." + module1 + ".trustStoreReloadIntervalMs", + Long.toString(module1ReloadIntervalMs)); + conf.set("celeborn.ssl.trustStoreReloadIntervalMs", Long.toString(defaultReloadIntervalMs)); + + TransportConf module1TestConf = new TransportConf(module1, conf); + TransportConf module2TestConf = new TransportConf(module2, conf); + + assertEquals(module1Protocol, module1TestConf.sslProtocol()); + assertEquals(module2Protocol, module2TestConf.sslProtocol()); + + assertEquals(module1ReloadIntervalMs, module1TestConf.sslTrustStoreReloadIntervalMs()); + assertEquals(defaultReloadIntervalMs, module2TestConf.sslTrustStoreReloadIntervalMs()); + } +} diff --git a/common/src/test/resources/ssl/generate_certs.sh b/common/src/test/resources/ssl/generate_certs.sh new file mode 100755 index 0000000000..d83cc7bc9a --- /dev/null +++ b/common/src/test/resources/ssl/generate_certs.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash +# 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. + + +# A simple utility to (re-)generate the files within resources/ssl +# These generated certificates are used for a variety of test scenarios for SSL. +# The utility ends up generating two certificates - which are saved into two different keystores +# The certificates generated are signed by two different CA cert's (also generated here). +# There are two truststores generated - the first truststore has both CA certs as part of it +# Hence this trust can be used to validate both client certificates. +# The second trust store has NO CA certs in it - and so when used will fail both the certificates. +# Requires: "openssl" (typically the openssl package) and java "keytool" in the PATH + +function gen_certs() { + + openssl genrsa -out ca.key 2048 + openssl req -x509 -new -days 9000 -key ca.key -out ca.crt -subj "/C=US/ST=YourState/L=YourCity/O=YourOrganization/CN=MyCACert" + openssl genrsa -out server.key 2048 + openssl req -new -key server.key -out server.csr -subj "/C=US/ST=YourState/L=YourCity/O=YourOrganization/CN=MyServer" + openssl x509 -req -CA ca.crt -CAkey ca.key -CAcreateserial -in server.csr -out server.crt -days 8000 + openssl pkcs12 -export -in server.crt -inkey server.key -out keystore.p12 -name servercert -password pass:password + keytool -importkeystore -destkeystore server.jks -srckeystore keystore.p12 -srcstoretype PKCS12 -deststoretype pkcs12 -srcstorepass password -deststorepass password -noprompt + + keytool -import -trustcacerts -alias CACert -file ca.crt -keystore truststore.jks -storepass password -noprompt + + rm ca.srl keystore.p12 server.csr ca.key server.key server.crt +} + + +mkdir for_default +cd for_default +gen_certs +cd .. +mkdir for_secondary +cd for_secondary +gen_certs +cd .. + + +cp ./for_default/truststore.jks ./for_default/server.jks . +cp ./for_secondary/server.jks ./server_another.jks + + +keytool -import -trustcacerts -alias 'CACertAnother' -file for_secondary/ca.crt -keystore ./truststore.jks -storepass password -noprompt + +# Copy the secondary trust store and remove the ca to generate truststore-without-ca.jks +cp ./for_secondary/truststore.jks ./truststore-without-ca.jks +keytool -delete -alias 'CACert' -keystore ./truststore-without-ca.jks -storepass password -noprompt + +rm -rf for_default for_secondary diff --git a/common/src/test/resources/ssl/server.jks b/common/src/test/resources/ssl/server.jks new file mode 100644 index 0000000000..164c9387a6 Binary files /dev/null and b/common/src/test/resources/ssl/server.jks differ diff --git a/common/src/test/resources/ssl/server_another.jks b/common/src/test/resources/ssl/server_another.jks new file mode 100644 index 0000000000..68b1cee5fa Binary files /dev/null and b/common/src/test/resources/ssl/server_another.jks differ diff --git a/common/src/test/resources/ssl/truststore-without-ca.jks b/common/src/test/resources/ssl/truststore-without-ca.jks new file mode 100644 index 0000000000..65d4b65283 Binary files /dev/null and b/common/src/test/resources/ssl/truststore-without-ca.jks differ diff --git a/common/src/test/resources/ssl/truststore.jks b/common/src/test/resources/ssl/truststore.jks new file mode 100644 index 0000000000..34eebe1099 Binary files /dev/null and b/common/src/test/resources/ssl/truststore.jks differ diff --git a/dev/dependencies.sh b/dev/dependencies.sh index 5da33d71a5..11fada9efb 100755 --- a/dev/dependencies.sh +++ b/dev/dependencies.sh @@ -250,6 +250,7 @@ if [ "$REPLACE" == "true" ]; then exit 0 fi -if [ "$CHECK" == "true" ]; then +# skip check for sbt server now until sbt_build_server_classpath is refactored +if [ "$CHECK" == "true" -a "$MODULE" != "server" ]; then check_diff fi diff --git a/dev/deps/dependencies-client-flink-1.14 b/dev/deps/dependencies-client-flink-1.14 index 1c6256450e..f0f24cf3f5 100644 --- a/dev/deps/dependencies-client-flink-1.14 +++ b/dev/deps/dependencies-client-flink-1.14 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar @@ -77,8 +77,8 @@ ratis-client/2.5.1//ratis-client-2.5.1.jar ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar diff --git a/dev/deps/dependencies-client-flink-1.15 b/dev/deps/dependencies-client-flink-1.15 index 1c6256450e..f0f24cf3f5 100644 --- a/dev/deps/dependencies-client-flink-1.15 +++ b/dev/deps/dependencies-client-flink-1.15 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar @@ -77,8 +77,8 @@ ratis-client/2.5.1//ratis-client-2.5.1.jar ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar diff --git a/dev/deps/dependencies-client-flink-1.17 b/dev/deps/dependencies-client-flink-1.17 index 1c6256450e..f0f24cf3f5 100644 --- a/dev/deps/dependencies-client-flink-1.17 +++ b/dev/deps/dependencies-client-flink-1.17 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar @@ -77,8 +77,8 @@ ratis-client/2.5.1//ratis-client-2.5.1.jar ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar diff --git a/dev/deps/dependencies-client-flink-1.18 b/dev/deps/dependencies-client-flink-1.18 index 1c6256450e..f0f24cf3f5 100644 --- a/dev/deps/dependencies-client-flink-1.18 +++ b/dev/deps/dependencies-client-flink-1.18 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar @@ -77,8 +77,8 @@ ratis-client/2.5.1//ratis-client-2.5.1.jar ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar diff --git a/dev/deps/dependencies-client-flink-1.19 b/dev/deps/dependencies-client-flink-1.19 index 1c6256450e..f0f24cf3f5 100644 --- a/dev/deps/dependencies-client-flink-1.19 +++ b/dev/deps/dependencies-client-flink-1.19 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar @@ -77,8 +77,8 @@ ratis-client/2.5.1//ratis-client-2.5.1.jar ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar diff --git a/dev/deps/dependencies-client-mr b/dev/deps/dependencies-client-mr index 010599e0be..498905f67d 100644 --- a/dev/deps/dependencies-client-mr +++ b/dev/deps/dependencies-client-mr @@ -41,10 +41,10 @@ curator-framework/5.2.0//curator-framework-5.2.0.jar curator-recipes/5.2.0//curator-recipes-5.2.0.jar dnsjava/2.1.7//dnsjava-2.1.7.jar ehcache/3.3.1//ehcache-3.3.1.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar geronimo-jcache_1.0_spec/1.0-alpha-1//geronimo-jcache_1.0_spec-1.0-alpha-1.jar gson/2.9.0//gson-2.9.0.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar guice-servlet/4.0//guice-servlet-4.0.jar guice/4.0//guice-4.0.jar hadoop-annotations/3.3.6//hadoop-annotations-3.3.6.jar @@ -186,8 +186,8 @@ ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar re2j/1.1//re2j-1.1.jar reload4j/1.2.22//reload4j-1.2.22.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar slf4j-reload4j/1.7.36//slf4j-reload4j-1.7.36.jar diff --git a/dev/deps/dependencies-client-spark-2.4 b/dev/deps/dependencies-client-spark-2.4 index 1d565ff7ee..9b044b54e5 100644 --- a/dev/deps/dependencies-client-spark-2.4 +++ b/dev/deps/dependencies-client-spark-2.4 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.0 b/dev/deps/dependencies-client-spark-3.0 index 0f15558228..b69c67e12f 100644 --- a/dev/deps/dependencies-client-spark-3.0 +++ b/dev/deps/dependencies-client-spark-3.0 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.1 b/dev/deps/dependencies-client-spark-3.1 index e58c2b7512..992d3365d1 100644 --- a/dev/deps/dependencies-client-spark-3.1 +++ b/dev/deps/dependencies-client-spark-3.1 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.2 b/dev/deps/dependencies-client-spark-3.2 index a4e18920b1..0ec524f543 100644 --- a/dev/deps/dependencies-client-spark-3.2 +++ b/dev/deps/dependencies-client-spark-3.2 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.3 b/dev/deps/dependencies-client-spark-3.3 index 1c6256450e..cb6f91f500 100644 --- a/dev/deps/dependencies-client-spark-3.3 +++ b/dev/deps/dependencies-client-spark-3.3 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.4 b/dev/deps/dependencies-client-spark-3.4 index fd38d5bb5f..bff920f8ad 100644 --- a/dev/deps/dependencies-client-spark-3.4 +++ b/dev/deps/dependencies-client-spark-3.4 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-client-spark-3.5 b/dev/deps/dependencies-client-spark-3.5 index 27292934b6..0232256581 100644 --- a/dev/deps/dependencies-client-spark-3.5 +++ b/dev/deps/dependencies-client-spark-3.5 @@ -20,8 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar diff --git a/dev/deps/dependencies-server b/dev/deps/dependencies-server index c0f7f0a26c..a09ea318c6 100644 --- a/dev/deps/dependencies-server +++ b/dev/deps/dependencies-server @@ -17,23 +17,55 @@ HikariCP/4.0.3//HikariCP-4.0.3.jar RoaringBitmap/0.9.32//RoaringBitmap-0.9.32.jar +aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar ap-loader-all/3.0-8//ap-loader-all-3.0-8.jar +classgraph/4.8.138//classgraph-4.8.138.jar commons-cli/1.5.0//commons-cli-1.5.0.jar commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -failureaccess/1.0.1//failureaccess-1.0.1.jar -guava/32.1.3-jre//guava-32.1.3-jre.jar +failureaccess/1.0.2//failureaccess-1.0.2.jar +guava/33.1.0-jre//guava-33.1.0-jre.jar hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar +hk2-api/2.6.1//hk2-api-2.6.1.jar +hk2-locator/2.6.1//hk2-locator-2.6.1.jar +hk2-utils/2.6.1//hk2-utils-2.6.1.jar jackson-annotations/2.15.3//jackson-annotations-2.15.3.jar jackson-core/2.15.3//jackson-core-2.15.3.jar jackson-databind/2.15.3//jackson-databind-2.15.3.jar +jackson-dataformat-yaml/2.13.2//jackson-dataformat-yaml-2.13.2.jar +jackson-datatype-jsr310/2.13.2//jackson-datatype-jsr310-2.13.2.jar +jackson-jaxrs-base/2.13.2//jackson-jaxrs-base-2.13.2.jar +jackson-jaxrs-json-provider/2.13.2//jackson-jaxrs-json-provider-2.13.2.jar +jackson-module-jaxb-annotations/2.14.1//jackson-module-jaxb-annotations-2.14.1.jar jackson-module-scala_2.12/2.15.3//jackson-module-scala_2.12-2.15.3.jar -javassist/3.28.0-GA//javassist-3.28.0-GA.jar -javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar +jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar +jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.servlet-api/4.0.4//jakarta.servlet-api-4.0.4.jar +jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar +jakarta.xml.bind-api/2.3.3//jakarta.xml.bind-api-2.3.3.jar +javassist/3.29.0-GA//javassist-3.29.0-GA.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar +jersey-client/2.39.1//jersey-client-2.39.1.jar +jersey-common/2.39.1//jersey-common-2.39.1.jar +jersey-container-servlet-core/2.39.1//jersey-container-servlet-core-2.39.1.jar +jersey-entity-filtering/2.39.1//jersey-entity-filtering-2.39.1.jar +jersey-hk2/2.39.1//jersey-hk2-2.39.1.jar +jersey-media-json-jackson/2.39.1//jersey-media-json-jackson-2.39.1.jar +jersey-media-multipart/2.39.1//jersey-media-multipart-2.39.1.jar +jersey-server/2.39.1//jersey-server-2.39.1.jar +jetty-client/9.4.52.v20230823//jetty-client-9.4.52.v20230823.jar +jetty-http/9.4.52.v20230823//jetty-http-9.4.52.v20230823.jar +jetty-io/9.4.52.v20230823//jetty-io-9.4.52.v20230823.jar +jetty-proxy/9.4.52.v20230823//jetty-proxy-9.4.52.v20230823.jar +jetty-security/9.4.52.v20230823//jetty-security-9.4.52.v20230823.jar +jetty-server/9.4.52.v20230823//jetty-server-9.4.52.v20230823.jar +jetty-servlet/9.4.52.v20230823//jetty-servlet-9.4.52.v20230823.jar +jetty-util-ajax/9.4.52.v20230823//jetty-util-ajax-9.4.52.v20230823.jar +jetty-util/9.4.52.v20230823//jetty-util-9.4.52.v20230823.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar @@ -46,6 +78,7 @@ maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar +mimepull/1.9.15//mimepull-1.9.15.jar mybatis/3.5.15//mybatis-3.5.15.jar netty-all/4.1.107.Final//netty-all-4.1.107.Final.jar netty-buffer/4.1.107.Final//netty-buffer-4.1.107.Final.jar @@ -81,6 +114,7 @@ netty-transport-rxtx/4.1.107.Final//netty-transport-rxtx-4.1.107.Final.jar netty-transport-sctp/4.1.107.Final//netty-transport-sctp-4.1.107.Final.jar netty-transport-udt/4.1.107.Final//netty-transport-udt-4.1.107.Final.jar netty-transport/4.1.107.Final//netty-transport-4.1.107.Final.jar +osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar protobuf-java/3.21.7//protobuf-java-3.21.7.jar ratis-client/2.5.1//ratis-client-2.5.1.jar @@ -95,10 +129,16 @@ ratis-shell/2.5.1//ratis-shell-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar reflections/0.10.2//reflections-0.10.2.jar rocksdbjni/8.11.3//rocksdbjni-8.11.3.jar -scala-library/2.12.15//scala-library-2.12.15.jar -scala-reflect/2.12.15//scala-reflect-2.12.15.jar +scala-library/2.12.18//scala-library-2.12.18.jar +scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/2.2//snakeyaml-2.2.jar snappy-java/1.1.10.5//snappy-java-1.1.10.5.jar +swagger-annotations/2.2.1//swagger-annotations-2.2.1.jar +swagger-core/2.2.1//swagger-core-2.2.1.jar +swagger-integration/2.2.1//swagger-integration-2.2.1.jar +swagger-jaxrs2/2.2.1//swagger-jaxrs2-2.2.1.jar +swagger-models/2.2.1//swagger-models-2.2.1.jar +swagger-ui/4.9.1//swagger-ui-4.9.1.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/merge_pr.py b/dev/merge_pr.py index f46370d595..4794e62aa4 100755 --- a/dev/merge_pr.py +++ b/dev/merge_pr.py @@ -64,8 +64,8 @@ GITHUB_OAUTH_KEY = os.environ.get("GITHUB_OAUTH_KEY") -GITHUB_BASE = "https://github.com/apache/incubator-celeborn/pull" -GITHUB_API_BASE = "https://api.github.com/repos/apache/incubator-celeborn" +GITHUB_BASE = "https://github.com/apache/celeborn/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/celeborn" JIRA_BASE = "https://issues.apache.org/jira/browse" JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches diff --git a/docs/README.md b/docs/README.md index 24b3ffa6b8..348318b0b0 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,7 +20,7 @@ license: | --- Quick Start === -This documentation gives a quick start guide for running Spark/Flink/MapReduce with Apache Celebornโ„ข(Incubating). +This documentation gives a quick start guide for running Spark/Flink/MapReduce with Apache Celebornโ„ข. ### Download Celeborn Download the latest Celeborn binary from the [Downloading Page](https://celeborn.apache.org/download/). diff --git a/docs/configuration/client.md b/docs/configuration/client.md index 20aa715807..761b481380 100644 --- a/docs/configuration/client.md +++ b/docs/configuration/client.md @@ -36,10 +36,8 @@ license: | | celeborn.client.flink.compression.enabled | true | false | Whether to compress data in Flink plugin. | 0.3.0 | remote-shuffle.job.enable-data-compression | | celeborn.client.flink.inputGate.concurrentReadings | 2147483647 | false | Max concurrent reading channels for a input gate. | 0.3.0 | remote-shuffle.job.concurrent-readings-per-gate | | celeborn.client.flink.inputGate.memory | 32m | false | Memory reserved for a input gate. | 0.3.0 | remote-shuffle.job.memory-per-gate | -| celeborn.client.flink.inputGate.minMemory | 8m | false | Min memory reserved for a input gate. | 0.3.0 | remote-shuffle.job.min.memory-per-gate | | celeborn.client.flink.inputGate.supportFloatingBuffer | true | false | Whether to support floating buffer in Flink input gates. | 0.3.0 | remote-shuffle.job.support-floating-buffer-per-input-gate | | celeborn.client.flink.resultPartition.memory | 64m | false | Memory reserved for a result partition. | 0.3.0 | remote-shuffle.job.memory-per-partition | -| celeborn.client.flink.resultPartition.minMemory | 8m | false | Min memory reserved for a result partition. | 0.3.0 | remote-shuffle.job.min.memory-per-partition | | celeborn.client.flink.resultPartition.supportFloatingBuffer | true | false | Whether to support floating buffer for result partitions. | 0.3.0 | remote-shuffle.job.support-floating-buffer-per-output-gate | | celeborn.client.mr.pushData.max | 32m | false | Max size for a push data sent from mr client. | 0.4.0 | | | celeborn.client.push.buffer.initial.size | 8k | false | | 0.3.0 | celeborn.push.buffer.initial.size | diff --git a/docs/configuration/master.md b/docs/configuration/master.md index 7a504441dc..e2217868b9 100644 --- a/docs/configuration/master.md +++ b/docs/configuration/master.md @@ -43,7 +43,9 @@ license: | | celeborn.master.heartbeat.worker.timeout | 120s | false | Worker heartbeat timeout. | 0.3.0 | celeborn.worker.heartbeat.timeout | | celeborn.master.host | <localhost> | false | Hostname for master to bind. | 0.2.0 | | | celeborn.master.http.host | <localhost> | false | Master's http host. | 0.4.0 | celeborn.metrics.master.prometheus.host,celeborn.master.metrics.prometheus.host | +| celeborn.master.http.maxWorkerThreads | 200 | false | Maximum number of threads in the master http worker thread pool. | 0.5.0 | | | celeborn.master.http.port | 9098 | false | Master's http port. | 0.4.0 | celeborn.metrics.master.prometheus.port,celeborn.master.metrics.prometheus.port | +| celeborn.master.http.stopTimeout | 5s | false | Master http server stop timeout. | 0.5.0 | | | celeborn.master.internal.port | 8097 | false | Internal port on the master where both workers and other master nodes connect. | 0.5.0 | | | celeborn.master.port | 9097 | false | Port for master to bind. | 0.2.0 | | | celeborn.master.rackResolver.refresh.interval | 30s | false | Interval for refreshing the node rack information periodically. | 0.5.0 | | diff --git a/docs/configuration/network.md b/docs/configuration/network.md index 35e438b6b4..a295d5353d 100644 --- a/docs/configuration/network.md +++ b/docs/configuration/network.md @@ -53,4 +53,13 @@ license: | | 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.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 | | +| celeborn.ssl.<module>.enabled | false | false | Enables SSL for securing wire traffic. | 0.5.0 | | +| celeborn.ssl.<module>.enabledAlgorithms | <undefined> | false | A comma-separated list of ciphers. The specified ciphers must be supported by JVM. The reference list of protocols can be found in the "JSSE Cipher Suite Names" section of the Java security guide. The list for Java 17 can be found at https://docs.oracle.com/en/java/javase/17/docs/specs/security/standard-names.html#jsse-cipher-suite-names . Note: If not set, the default cipher suite for the JRE will be used. | 0.5.0 | | +| celeborn.ssl.<module>.keyStore | <undefined> | false | Path to the key store file. The path can be absolute or relative to the directory in which the process is started. | 0.5.0 | | +| celeborn.ssl.<module>.keyStorePassword | <undefined> | false | Password to the key store. | 0.5.0 | | +| celeborn.ssl.<module>.protocol | TLSv1.2 | false | SSL protocol to use | 0.5.0 | | +| celeborn.ssl.<module>.trustStore | <undefined> | false | Path to the trust store file. The path can be absolute or relative to the directory in which the process is started. | 0.5.0 | | +| celeborn.ssl.<module>.trustStorePassword | <undefined> | false | Password for the trust store. | 0.5.0 | | +| celeborn.ssl.<module>.trustStoreReloadIntervalMs | 10s | false | The interval at which the trust store should be reloaded (in milliseconds). This setting is mostly only useful for server components, not applications. | 0.5.0 | | +| celeborn.ssl.<module>.trustStoreReloadingEnabled | false | false | Whether the trust store should be reloaded periodically. This setting is mostly only useful for server components, not applications. | 0.5.0 | | diff --git a/docs/configuration/worker.md b/docs/configuration/worker.md index a35e35b8ce..38b86b5187 100644 --- a/docs/configuration/worker.md +++ b/docs/configuration/worker.md @@ -83,7 +83,9 @@ license: | | celeborn.worker.graceful.shutdown.saveCommittedFileInfo.sync | false | false | Whether to call sync method to save committed file infos into Level DB to handle OS crash. | 0.3.1 | | | celeborn.worker.graceful.shutdown.timeout | 600s | false | The worker's graceful shutdown timeout time. | 0.2.0 | | | celeborn.worker.http.host | <localhost> | false | Worker's http host. | 0.4.0 | celeborn.metrics.worker.prometheus.host,celeborn.worker.metrics.prometheus.host | +| celeborn.worker.http.maxWorkerThreads | 200 | false | Maximum number of threads in the worker http worker thread pool. | 0.5.0 | | | celeborn.worker.http.port | 9096 | false | Worker's http port. | 0.4.0 | celeborn.metrics.worker.prometheus.port,celeborn.worker.metrics.prometheus.port | +| celeborn.worker.http.stopTimeout | 5s | false | Worker http server stop timeout. | 0.5.0 | | | celeborn.worker.internal.port | 0 | false | Internal server port on the Worker where the master nodes connect. | 0.5.0 | | | celeborn.worker.jvmProfiler.enabled | false | false | Turn on code profiling via async_profiler in workers. | 0.5.0 | | | celeborn.worker.jvmProfiler.localDir | . | false | Local file system path on worker where profiler output is saved. Defaults to the working directory of the worker process. | 0.5.0 | | diff --git a/docs/developers/glutensupport.md b/docs/developers/glutensupport.md index 645c4af221..6092caec98 100644 --- a/docs/developers/glutensupport.md +++ b/docs/developers/glutensupport.md @@ -31,7 +31,7 @@ mvn clean package -Pbackends-velox -Pspark-3.3 -Prss -DskipTests Then add the Gluten and Spark Celeborn Client packages to your Spark application's classpath(usually add them into `$SPARK_HOME/jars`). -- Celeborn: `celeborn-client-spark-3-shaded_2.12-0.3.0-incubating.jar` +- Celeborn: `celeborn-client-spark-3-shaded_2.12-[celebornVersion].jar` - Gluten: `gluten-velox-bundle-spark3.x_2.12-xx-xx-SNAPSHOT.jar`, `gluten-thirdparty-lib-xx.jar` Currently, to use Gluten following configurations are required in `spark-defaults.conf`. @@ -52,7 +52,7 @@ spark.shuffle.service.enabled false spark.sql.adaptive.localShuffleReader.enabled false # If you want to use dynamic resource allocation, -# please refer to this URL (https://github.com/apache/incubator-celeborn/tree/main/assets/spark-patch) to apply the patch into your own Spark. +# please refer to this URL (https://github.com/apache/celeborn/tree/main/assets/spark-patch) to apply the patch into your own Spark. spark.dynamicAllocation.enabled false ``` diff --git a/docs/developers/overview.md b/docs/developers/overview.md index b9ff004318..1b05f0e35f 100644 --- a/docs/developers/overview.md +++ b/docs/developers/overview.md @@ -18,7 +18,7 @@ license: | # Celeborn Architecture -This article introduces high level Apache Celebornโ„ข(Incubating) Architecture. For more detailed description of each module/process, +This article introduces high level Apache Celebornโ„ข Architecture. For more detailed description of each module/process, please refer to dedicated articles. ## Why Celeborn @@ -30,13 +30,13 @@ the disk and network inefficiency (M * N between Mappers and Reducers) in tradit Besides inefficiency, traditional shuffle framework requires large local storage in compute node to store shuffle data, thus blocks the adoption of disaggregated architecture. -Apache Celeborn(Incubating) solves the problems by reorganizing shuffle data in a more efficient way, and storing the data in +Apache Celeborn solves the problems by reorganizing shuffle data in a more efficient way, and storing the data in a separate service. The high level architecture of Celeborn is as follows: ![Celeborn](../../assets/img/celeborn.svg) ## Components -Celeborn(Incubating) has three primary components: Master, Worker, and Client. +Celeborn has three primary components: Master, Worker, and Client. - Master manages Celeborn cluster and achieves high availability(HA) based on Raft. - Worker processes read-write requests. diff --git a/docs/migration.md b/docs/migration.md index 0c6214da23..93802dd52d 100644 --- a/docs/migration.md +++ b/docs/migration.md @@ -36,7 +36,7 @@ license: | - Since 0.5.0, Celeborn deprecate `celeborn.quota.configuration.path`. Please use `celeborn.dynamicConfig.store.fs.path` instead. -- Since 0.5.0, Celeborn client removes configuration `celeborn.client.push.splitPartition.threads`. +- Since 0.5.0, Celeborn client removes configuration `celeborn.client.push.splitPartition.threads`, `celeborn.client.flink.inputGate.minMemory` and `celeborn.client.flink.resultPartition.minMemory`. ## Upgrading from 0.4.0 to 0.4.1 diff --git a/master/pom.xml b/master/pom.xml index a5260a1cef..4dd3ed11c7 100644 --- a/master/pom.xml +++ b/master/pom.xml @@ -83,11 +83,27 @@ log4j-1.2-api + + org.apache.celeborn + celeborn-service_${scala.binary.version} + ${project.version} + test-jar + org.mockito mockito-core test + + org.glassfish.jersey.test-framework + jersey-test-framework-core + test + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-jetty + test + diff --git a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAHelper.java b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAHelper.java index fdd22443e8..8d2669f207 100644 --- a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAHelper.java +++ b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAHelper.java @@ -19,6 +19,9 @@ import java.io.File; import java.io.IOException; +import java.util.Optional; + +import scala.Tuple2; import com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.protocol.Message; @@ -50,11 +53,13 @@ public static void sendFailure( RpcCallContext context, HARaftServer ratisServer, Throwable cause) { if (context != null) { if (ratisServer != null) { - if (ratisServer.getCachedLeaderPeerRpcEndpoint().isPresent()) { + Optional> leaderPeer = ratisServer.getCachedLeaderPeerRpcEndpoint(); + if (leaderPeer.isPresent()) { context.sendFailure( new MasterNotLeaderException( ratisServer.getRpcEndpoint(), - ratisServer.getCachedLeaderPeerRpcEndpoint().get(), + leaderPeer.get()._1(), + leaderPeer.get()._2(), cause)); } else { context.sendFailure( diff --git a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HARaftServer.java b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HARaftServer.java index 9e502058f0..b9d690d337 100644 --- a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HARaftServer.java +++ b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HARaftServer.java @@ -74,6 +74,7 @@ static long nextCallId() { private final InetSocketAddress ratisAddr; private final String rpcEndpoint; + private final String internalRpcEndpoint; private final RaftServer server; private final RaftGroup raftGroup; private final RaftPeerId raftPeerId; @@ -89,7 +90,7 @@ static long nextCallId() { private long roleCheckIntervalMs; private final ReentrantReadWriteLock roleCheckLock = new ReentrantReadWriteLock(); private Optional cachedPeerRole = Optional.empty(); - private Optional cachedLeaderPeerRpcEndpoint = Optional.empty(); + private Optional> cachedLeaderPeerRpcEndpoints = Optional.empty(); private final CelebornConf conf; private long workerTimeoutDeadline; private long appTimeoutDeadline; @@ -109,11 +110,13 @@ private HARaftServer( RaftPeerId localRaftPeerId, InetSocketAddress ratisAddr, String rpcEndpoint, + String internalRpcEndpoint, List raftPeers) throws IOException { this.metaHandler = metaHandler; this.ratisAddr = ratisAddr; this.rpcEndpoint = rpcEndpoint; + this.internalRpcEndpoint = internalRpcEndpoint; this.raftPeerId = localRaftPeerId; this.raftGroup = RaftGroup.valueOf(RAFT_GROUP_ID, raftPeers); this.masterStateMachine = getStateMachine(); @@ -163,6 +166,8 @@ public static HARaftServer newMasterRatisServer( .setId(localRaftPeerId) .setAddress(ratisAddr) .setClientAddress(localNode.rpcEndpoint()) + // We use admin address to host the internal rpc address + .setAdminAddress(localNode.internalRpcEndpoint()) .build(); List raftPeers = new ArrayList<>(); // Add this Ratis server to the Ratis ring @@ -178,6 +183,8 @@ public static HARaftServer newMasterRatisServer( .setId(raftPeerId) .setAddress(peer.ratisEndpoint()) .setClientAddress(peer.rpcEndpoint()) + // We use admin address to host the internal rpc address + .setAdminAddress(peer.internalRpcEndpoint()) .build(); } else { InetSocketAddress peerRatisAddr = peer.ratisAddr(); @@ -186,6 +193,8 @@ public static HARaftServer newMasterRatisServer( .setId(raftPeerId) .setAddress(peerRatisAddr) .setClientAddress(peer.rpcEndpoint()) + // We use admin address to host the internal rpc address + .setAdminAddress(peer.internalRpcEndpoint()) .build(); } @@ -193,7 +202,13 @@ public static HARaftServer newMasterRatisServer( raftPeers.add(raftPeer); }); return new HARaftServer( - metaHandler, conf, localRaftPeerId, ratisAddr, localNode.rpcEndpoint(), raftPeers); + metaHandler, + conf, + localRaftPeerId, + ratisAddr, + localNode.rpcEndpoint(), + localNode.internalRpcEndpoint(), + raftPeers); } public ResourceResponse submitRequest(ResourceProtos.ResourceRequest request) @@ -421,12 +436,12 @@ public boolean isLeader() { /** * Get the suggested leader peer id. * - * @return RaftPeerId of the suggested leader node. + * @return RaftPeerId of the suggested leader node - Tuple2(rpc endpoint, internal rpc endpoint) */ - public Optional getCachedLeaderPeerRpcEndpoint() { + public Optional> getCachedLeaderPeerRpcEndpoint() { this.roleCheckLock.readLock().lock(); try { - return cachedLeaderPeerRpcEndpoint; + return cachedLeaderPeerRpcEndpoints; } finally { this.roleCheckLock.readLock().unlock(); } @@ -442,22 +457,30 @@ public void updateServerRole() { RaftProtos.RaftPeerRole thisNodeRole = roleInfoProto.getRole(); if (thisNodeRole.equals(RaftProtos.RaftPeerRole.LEADER)) { - setServerRole(thisNodeRole, getRpcEndpoint()); + setServerRole(thisNodeRole, getRpcEndpoint(), getInternalRpcEndpoint()); } else if (thisNodeRole.equals(RaftProtos.RaftPeerRole.FOLLOWER)) { ByteString leaderNodeId = roleInfoProto.getFollowerInfo().getLeaderInfo().getId().getId(); // There may be a chance, here we get leaderNodeId as null. For // example, in 3 node Ratis, if 2 nodes are down, there will // be no leader. String leaderPeerRpcEndpoint = null; + String leaderPeerInternalRpcEndpoint = null; if (leaderNodeId != null && !leaderNodeId.isEmpty()) { leaderPeerRpcEndpoint = roleInfoProto.getFollowerInfo().getLeaderInfo().getId().getClientAddress(); + // We use admin address to host the internal rpc address + if (conf.internalPortEnabled()) { + leaderPeerInternalRpcEndpoint = + roleInfoProto.getFollowerInfo().getLeaderInfo().getId().getAdminAddress(); + } else { + leaderPeerInternalRpcEndpoint = leaderPeerRpcEndpoint; + } } - setServerRole(thisNodeRole, leaderPeerRpcEndpoint); + setServerRole(thisNodeRole, leaderPeerRpcEndpoint, leaderPeerInternalRpcEndpoint); } else { - setServerRole(thisNodeRole, null); + setServerRole(thisNodeRole, null, null); } } catch (IOException e) { LOG.error( @@ -465,12 +488,15 @@ public void updateServerRole() { + "{} and resetting leader info.", RaftProtos.RaftPeerRole.UNRECOGNIZED, e); - setServerRole(null, null); + setServerRole(null, null, null); } } /** Set the current server role and the leader peer rpc endpoint. */ - private void setServerRole(RaftProtos.RaftPeerRole currentRole, String leaderPeerRpcEndpoint) { + private void setServerRole( + RaftProtos.RaftPeerRole currentRole, + String leaderPeerRpcEndpoint, + String leaderPeerInternalRpcEndpoint) { this.roleCheckLock.writeLock().lock(); try { boolean leaderChanged = false; @@ -490,7 +516,12 @@ private void setServerRole(RaftProtos.RaftPeerRole currentRole, String leaderPee } this.cachedPeerRole = Optional.ofNullable(currentRole); - this.cachedLeaderPeerRpcEndpoint = Optional.ofNullable(leaderPeerRpcEndpoint); + if (null != leaderPeerRpcEndpoint) { + this.cachedLeaderPeerRpcEndpoints = + Optional.of(Tuple2.apply(leaderPeerRpcEndpoint, leaderPeerInternalRpcEndpoint)); + } else { + this.cachedLeaderPeerRpcEndpoints = Optional.empty(); + } } finally { this.roleCheckLock.writeLock().unlock(); } @@ -510,6 +541,10 @@ public String getRpcEndpoint() { return this.rpcEndpoint; } + public String getInternalRpcEndpoint() { + return this.internalRpcEndpoint; + } + void stepDown() { try { TransferLeadershipRequest request = diff --git a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala index 4942dac871..05ae53cc3e 100644 --- a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala +++ b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala @@ -78,6 +78,9 @@ private[celeborn] class Master( private val authEnabled = conf.authEnabled private val secretRegistry = new MasterSecretRegistryImpl() + private val sendApplicationMetaThreads = conf.masterSendApplicationMetaThreads + // Send ApplicationMeta to workers + private var sendApplicationMetaExecutor: ExecutorService = _ override val rpcEnv: RpcEnv = if (!authEnabled) { @@ -258,9 +261,6 @@ private[celeborn] class Master( internalRpcEndpoint) } - private val sendApplicationMetaThreads = conf.masterSendApplicationMetaThreads - // Send ApplicationMeta to workers - private var sendApplicationMetaExecutor: ExecutorService = _ // Maintains the mapping for the workers assigned to each application private val workersAssignedToApp : util.concurrent.ConcurrentHashMap[String, util.Set[WorkerInfo]] = diff --git a/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterClusterInfo.scala b/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterClusterInfo.scala index a661c6acaf..1042f46273 100644 --- a/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterClusterInfo.scala +++ b/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterClusterInfo.scala @@ -43,7 +43,9 @@ object MasterClusterInfo extends Logging { val ratisPort = conf.haMasterRatisPort(nodeId) val rpcHost = conf.haMasterNodeHost(nodeId) val rpcPort = conf.haMasterNodePort(nodeId) - MasterNode(nodeId, ratisHost, ratisPort, rpcHost, rpcPort) + val internalPort = + if (conf.internalPortEnabled) conf.haMasterNodeInternalPort(nodeId) else rpcPort + MasterNode(nodeId, ratisHost, ratisPort, rpcHost, rpcPort, internalPort) } val (localNodes, peerNodes) = localNodeIdOpt match { diff --git a/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterNode.scala b/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterNode.scala index a1ac2b67ea..b60b76eda5 100644 --- a/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterNode.scala +++ b/master/src/main/scala/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterNode.scala @@ -29,7 +29,8 @@ case class MasterNode( ratisHost: String, ratisPort: Int, rpcHost: String, - rpcPort: Int) { + rpcPort: Int, + internalRpcPort: Int) { def isRatisHostUnresolved: Boolean = ratisAddr.isUnresolved @@ -39,6 +40,8 @@ case class MasterNode( def rpcEndpoint: String = rpcHost + ":" + rpcPort + def internalRpcEndpoint: String = rpcHost + ":" + internalRpcPort + lazy val ratisAddr = MasterNode.createSocketAddr(ratisHost, ratisPort) lazy val rpcAddr = MasterNode.createSocketAddr(rpcHost, rpcPort) @@ -52,6 +55,7 @@ object MasterNode extends Logging { private var ratisPort = 0 private var rpcHost: String = _ private var rpcPort = 0 + private var internalRpcPort = 0 def setNodeId(nodeId: String): this.type = { this.nodeId = nodeId @@ -84,7 +88,13 @@ object MasterNode extends Logging { this } - def build: MasterNode = MasterNode(nodeId, ratisHost, ratisPort, rpcHost, rpcPort) + def setInternalRpcPort(internalRpcPort: Int): this.type = { + this.internalRpcPort = internalRpcPort + this + } + + def build: MasterNode = + MasterNode(nodeId, ratisHost, ratisPort, rpcHost, rpcPort, internalRpcPort) } private def createSocketAddr(host: String, port: Int): InetSocketAddress = { diff --git a/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResource.scala b/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResource.scala new file mode 100644 index 0000000000..2093dee03d --- /dev/null +++ b/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResource.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.service.deploy.master.http.api + +import javax.ws.rs.{GET, Path, POST, QueryParam} +import javax.ws.rs.core.MediaType + +import io.swagger.v3.oas.annotations.media.Content +import io.swagger.v3.oas.annotations.responses.ApiResponse + +import org.apache.celeborn.server.common.http.api.ApiRequestContext + +@Path("/") +class ApiMasterResource extends ApiRequestContext { + + @Path("/masterGroupInfo") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "List master group information of the service. It will list all master's LEADER, FOLLOWER information.") + @GET + def masterGroupInfo: String = httpService.getMasterGroupInfo + + @Path("/lostWorkers") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all lost workers of the master.") + @GET + def lostWorkers: String = httpService.getLostWorkers + + @Path("/excludedWorkers") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all excluded workers of the master.") + @GET + def excludedWorkers: String = httpService.getExcludedWorkers + + @Path("/shutdownWorkers") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all shutdown workers of the master.") + @GET + def shutdownWorkers: String = httpService.getShutdownWorkers + + @Path("/hostnames") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all running application's LifecycleManager's hostnames of the cluster.") + @GET + def hostnames: String = httpService.getHostnameList + + @Path("/sendWorkerEvent") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "For Master(Leader) can send worker event to manager workers. Legal types are 'None', 'Immediately', 'Decommission', 'DecommissionThenIdle', 'Graceful', 'Recommission'") + @POST + def sendWorkerEvent( + @QueryParam("TYPE") eventType: String, + @QueryParam("WORKERS") workers: String): String = { + httpService.handleWorkerEvent(normalizeParam(eventType), normalizeParam(workers)) + } + + @Path("/workerEventInfo") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all worker event infos of the master.") + @GET + def workerEventInfo: String = httpService.getWorkerEventInfo() + + @Path("/exclude") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all worker event infos of the master.") + @POST + def excludeWorkers( + @QueryParam("ADD") addWorkers: String, + @QueryParam("REMOVE") removeWorkers: String): String = { + httpService.exclude(normalizeParam(addWorkers), normalizeParam(removeWorkers)) + } +} diff --git a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java index d7cf715a0f..9dbcff797d 100644 --- a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java +++ b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java @@ -25,6 +25,8 @@ import java.util.*; import java.util.concurrent.atomic.AtomicLong; +import scala.Tuple2; + import org.junit.*; import org.mockito.Mockito; @@ -127,6 +129,7 @@ public static void resetRaftServer() throws IOException, InterruptedException { .setHost(Utils.localHostName(conf1)) .setRatisPort(ratisPort1) .setRpcPort(ratisPort1) + .setInternalRpcPort(ratisPort1) .setNodeId(id1) .build(); MasterNode masterNode2 = @@ -134,6 +137,7 @@ public static void resetRaftServer() throws IOException, InterruptedException { .setHost(Utils.localHostName(conf2)) .setRatisPort(ratisPort2) .setRpcPort(ratisPort2) + .setInternalRpcPort(ratisPort2) .setNodeId(id2) .build(); MasterNode masterNode3 = @@ -141,6 +145,7 @@ public static void resetRaftServer() throws IOException, InterruptedException { .setHost(Utils.localHostName(conf3)) .setRatisPort(ratisPort3) .setRpcPort(ratisPort3) + .setInternalRpcPort(ratisPort3) .setNodeId(id3) .build(); @@ -179,6 +184,26 @@ public void testLeaderAvaiable() { boolean hasLeader = RATISSERVER1.isLeader() || RATISSERVER2.isLeader() || RATISSERVER3.isLeader(); Assert.assertTrue(hasLeader); + + // Check if the rpc endpoint and internal rpc endpoint of the leader is as expected. + + HARaftServer leader = + RATISSERVER1.isLeader() + ? RATISSERVER1 + : (RATISSERVER2.isLeader() ? RATISSERVER2 : RATISSERVER3); + // one of them must be the follower given the three servers we have + HARaftServer follower = RATISSERVER1.isLeader() ? RATISSERVER2 : RATISSERVER1; + + // This is expected to be false, but as a side effect, updates getCachedLeaderPeerRpcEndpoint + boolean isFollowerCurrentLeader = follower.isLeader(); + Assert.assertFalse(isFollowerCurrentLeader); + + Optional> cachedLeaderPeerRpcEndpoint = + follower.getCachedLeaderPeerRpcEndpoint(); + + Assert.assertTrue(cachedLeaderPeerRpcEndpoint.isPresent()); + Assert.assertEquals(leader.getRpcEndpoint(), cachedLeaderPeerRpcEndpoint.get()._1()); + Assert.assertEquals(leader.getInternalRpcEndpoint(), cachedLeaderPeerRpcEndpoint.get()._2()); } private static final String HOSTNAME1 = "host1"; diff --git a/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResourceSuite.scala b/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResourceSuite.scala new file mode 100644 index 0000000000..5c272446f9 --- /dev/null +++ b/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/ApiMasterResourceSuite.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.service.deploy.master.http.api + +import javax.ws.rs.core.MediaType + +import com.google.common.io.Files + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.util.{CelebornExitKind, Utils} +import org.apache.celeborn.server.common.HttpService +import org.apache.celeborn.server.common.http.ApiBaseResourceSuite +import org.apache.celeborn.service.deploy.master.{Master, MasterArguments} + +class ApiMasterResourceSuite extends ApiBaseResourceSuite { + private var master: Master = _ + + override protected def httpService: HttpService = master + + def getTmpDir(): String = { + val tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + tmpDir.getAbsolutePath + } + + override def beforeAll(): Unit = { + val randomMasterPort = Utils.selectRandomPort(1024, 65535) + val randomHttpPort = randomMasterPort + 1 + celebornConf.set(CelebornConf.HA_ENABLED.key, "false") + celebornConf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir()) + celebornConf.set(CelebornConf.WORKER_STORAGE_DIRS.key, getTmpDir()) + celebornConf.set(CelebornConf.MASTER_HTTP_HOST.key, "127.0.0.1") + celebornConf.set(CelebornConf.MASTER_HTTP_PORT.key, randomHttpPort.toString) + + val args = Array("-h", "localhost", "-p", randomMasterPort.toString) + + val masterArgs = new MasterArguments(args, celebornConf) + master = new Master(celebornConf, masterArgs) + new Thread() { + override def run(): Unit = { + master.initialize() + } + }.start() + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + master.stop(CelebornExitKind.EXIT_IMMEDIATELY) + master.rpcEnv.shutdown() + } + + test("masterGroupInfo") { + val response = webTarget.path("masterGroupInfo").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("lostWorkers") { + val response = webTarget.path("lostWorkers").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("excludedWorkers") { + val response = webTarget.path("excludedWorkers").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("shutdownWorkers") { + val response = webTarget.path("shutdownWorkers").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("hostnames") { + val response = webTarget.path("hostnames").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("sendWorkerEvent") { + val response = webTarget.path("sendWorkerEvent") + .request(MediaType.TEXT_PLAIN) + .post(null) + assert(200 == response.getStatus) + } + + test("workerEventInfo") { + val response = webTarget.path("workerEventInfo").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("exclude") { + val response = webTarget.path("exclude").request(MediaType.TEXT_PLAIN).post(null) + assert(200 == response.getStatus) + } +} diff --git a/mkdocs.yml b/mkdocs.yml index cc4adfe49b..8318f8d7c4 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -15,9 +15,9 @@ # limitations under the License. # -site_name: Apache Celebornโ„ข (Incubating) -repo_name: apache/incubator-celeborn -repo_url: https://gitbox.apache.org/repos/asf/incubator-celeborn.git +site_name: Apache Celebornโ„ข +repo_name: apache/celeborn +repo_url: https://gitbox.apache.org/repos/asf/celeborn.git plugins: - search @@ -53,20 +53,11 @@ extra: - icon: fontawesome/brands/github copyright: > - ApacheCon North America -
- Copyright ยฉ 2022 The Apache Software Foundation + Copyright ยฉ 2022-2024 The Apache Software Foundation, Licensed under the Apache License, Version 2.0. Privacy Policy

- Apache Celebornโ„ข, Apache Incubator, Apache, the Apache feather logo, and the Apache Incubator project logo are - trademarks or registered trademarks of The Apache Software Foundation.
-
- Apache Celebornโ„ข is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the - Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that - the infrastructure, communications, and decision making process have stabilized in a manner consistent with - other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or - stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
+ Apache Celebornโ„ข, Apache, and the Apache feather logo are trademarks or registered trademarks of The Apache Software Foundation.

Please visit
Apache Software Foundation for more details.

diff --git a/pom.xml b/pom.xml index f5825ed377..af1af9a321 100644 --- a/pom.xml +++ b/pom.xml @@ -58,11 +58,11 @@ UTF-8 8 - 2.12.15 + 2.12.18 2.12 ${java.version} ${java.version} - 3.8.8 + 3.9.6 1.14.6 3.3.4 @@ -79,8 +79,7 @@ 9+181-r4173-1 1.3.9 1.44.0 - 32.1.3-jre - 3.1.0 + 33.1.0-jre 4.13.2 1.8 2.17.2 @@ -88,6 +87,7 @@ 4.11.0 1.17.14 4.1.107.Final + 1.77 3.21.7 2.5.1 3.2.16 @@ -106,21 +106,28 @@ 4.0.3 2.2.224 + + 2.2.1 + 4.9.1 + 2.39.1 + 9.4.52.v20230823 + 4.0.4 + org.apache.celeborn.shaded 3.0.0 3.2.0 - 3.11.0 + 3.12.1 3.1.1 1.3.0 - 0.8.7 + 0.8.11 3.0.2 1.7.0 0.6.1 0.13 4.7.2 2.1.0 - 3.4.0 + 3.5.2 2.24.1 3.0.0-M7 1.7.13 @@ -354,11 +361,6 @@ protobuf-java ${protobuf.version} - - javax.servlet - javax.servlet-api - ${javaxservlet.version} - io.netty netty-all @@ -456,6 +458,105 @@ test + + + jakarta.servlet + jakarta.servlet-api + ${jakarta.servlet-api.version} + + + + org.glassfish.jersey.core + jersey-server + ${jersey.version} + + + jakarta.xml.bind + jakarta.xml.bind-api + + + + + + org.glassfish.jersey.containers + jersey-container-servlet-core + ${jersey.version} + + + + org.glassfish.jersey.inject + jersey-hk2 + ${jersey.version} + + + + org.glassfish.jersey.media + jersey-media-json-jackson + ${jersey.version} + + + + org.glassfish.jersey.media + jersey-media-multipart + ${jersey.version} + + + + org.glassfish.jersey.test-framework + jersey-test-framework-core + ${jersey.version} + + + jakarta.activation + jakarta.activation-api + + + + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-jetty + ${jersey.version} + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-continuation + + + + + + io.swagger.core.v3 + swagger-jaxrs2 + ${swagger.version} + + + com.sun.activation + jakarta.activation + + + org.javassist + javassist + + + + + + + org.webjars + swagger-ui + ${swagger-ui.version} + + junit junit @@ -522,6 +623,19 @@ snappy-java ${snappy.version} + + + org.bouncycastle + bcprov-jdk18on + ${bouncycastle.version} + test + + + org.bouncycastle + bcpkix-jdk18on + ${bouncycastle.version} + test + @@ -1312,6 +1426,62 @@
+ + jdk-21 + + 21 + + + 21 + 2.25.0 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven.plugin.compiler.version} + + ${java.version} + ${java.version} + UTF-8 + + -XDcompilePolicy=simple + -Xplugin:ErrorProne \ + -XepExcludedPaths:.*/target/generated-sources/protobuf/.* \ + -Xep:FutureReturnValueIgnored:OFF \ + -Xep:TypeParameterUnusedInFormals:OFF \ + -Xep:UnusedVariable:OFF \ + -Xep:StringSplitter:OFF \ + -Xep:EmptyBlockTag:OFF \ + -Xep:EqualsGetClass:OFF \ + -Xep:MissingSummary:OFF \ + -Xep:BadImport:OFF + -J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED + -J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED + -J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED + -J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED + + + + com.google.errorprone + error_prone_core + ${error-prone.version} + + + + + + + + flink-1.14 diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index fc7052a36c..c9d11acefe 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -43,9 +43,8 @@ object Dependencies { val commonsLoggingVersion = "1.1.3" val commonsLang3Version = "3.12.0" val findbugsVersion = "1.3.9" - val guavaVersion = "32.1.3-jre" + val guavaVersion = "33.1.0-jre" val hadoopVersion = "3.3.6" - val javaxServletVersion = "3.1.0" val junitInterfaceVersion = "0.13.3" // don't forget update `junitInterfaceVersion` when we upgrade junit val junitVersion = "4.13.2" @@ -67,6 +66,14 @@ object Dependencies { val mybatisVersion = "3.5.15" val hikaricpVersion = "4.0.3" val h2Version = "2.2.224" + val swaggerVersion = "2.2.1" + val swaggerUiVersion = "4.9.1" + val jerseyVersion = "2.39.1" + val jettyVersion = "9.4.52.v20230823" + val jakartaServeletApiVersion = "4.0.4" + + // For SSL support + val bouncycastleVersion = "1.77" // Versions for proto val protocVersion = "3.21.7" @@ -102,7 +109,6 @@ object Dependencies { val ioDropwizardMetricsJvm = "io.dropwizard.metrics" % "metrics-jvm" % metricsVersion val ioNetty = "io.netty" % "netty-all" % nettyVersion excludeAll( ExclusionRule("io.netty", "netty-handler-ssl-ocsp")) - val javaxServletApi = "javax.servlet" % "javax.servlet-api" % javaxServletVersion val leveldbJniAll = "org.fusesource.leveldbjni" % "leveldbjni-all" % leveldbJniVersion val log4j12Api = "org.apache.logging.log4j" % "log4j-1.2-api" % log4j2Version val log4jSlf4jImpl = "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4j2Version @@ -130,6 +136,23 @@ object Dependencies { val zstdJni = "com.github.luben" % "zstd-jni" % zstdJniVersion val mybatis = "org.mybatis" % "mybatis" % mybatisVersion val hikaricp = "com.zaxxer" % "HikariCP" % hikaricpVersion + val jettyServer = "org.eclipse.jetty" % "jetty-server" % jettyVersion excludeAll( + ExclusionRule("javax.servlet", "javax.servlet-api")) + val jettyServlet = "org.eclipse.jetty" % "jetty-servlet" % jettyVersion excludeAll( + ExclusionRule("javax.servlet", "javax.servlet-api")) + val jettyProxy = "org.eclipse.jetty" % "jetty-proxy" % jettyVersion + val jakartaServletApi = "jakarta.servlet" % "jakarta.servlet-api" % jakartaServeletApiVersion + val jerseyServer = "org.glassfish.jersey.core" % "jersey-server" % jerseyVersion excludeAll( + ExclusionRule("jakarta.xml.bind", "jakarta.xml.bind-api")) + val jerseyContainerServletCore = "org.glassfish.jersey.containers" % "jersey-container-servlet-core" % jerseyVersion + val jerseyHk2 = "org.glassfish.jersey.inject" % "jersey-hk2" % jerseyVersion + val jerseyMediaJsonJackson = "org.glassfish.jersey.media" % "jersey-media-json-jackson" % jerseyVersion + val jerseyMediaMultipart = "org.glassfish.jersey.media" % "jersey-media-multipart" % jerseyVersion + val swaggerJaxrs2 = "io.swagger.core.v3" % "swagger-jaxrs2" %swaggerVersion excludeAll( + ExclusionRule("com.sun.activation", "jakarta.activation"), + ExclusionRule("org.javassist", "javassist"), + ExclusionRule("jakarta.activation", "jakarta.activation-api")) + val swaggerUi = "org.webjars" % "swagger-ui" % swaggerUiVersion // Test dependencies // https://www.scala-sbt.org/1.x/docs/Testing.html @@ -140,6 +163,14 @@ object Dependencies { val scalatestMockito = "org.mockito" %% "mockito-scala-scalatest" % scalatestMockitoVersion val scalatest = "org.scalatest" %% "scalatest" % scalatestVersion val h2 = "com.h2database" % "h2" % h2Version + val jerseyTestFrameworkCore = "org.glassfish.jersey.test-framework" % "jersey-test-framework-core" % jerseyVersion + val jerseyTestFrameworkProviderJetty = "org.glassfish.jersey.test-framework.providers" % "jersey-test-framework-provider-jetty" % jerseyVersion excludeAll( + ExclusionRule("org.eclipse.jetty", "jetty-util"), + ExclusionRule("org.eclipse.jetty", "jetty-continuation")) + + // SSL support + val bouncycastleBcprovJdk18on = "org.bouncycastle" % "bcprov-jdk18on" % bouncycastleVersion % "test" + val bouncycastleBcpkixJdk18on = "org.bouncycastle" % "bcpkix-jdk18on" % bouncycastleVersion % "test" } object CelebornCommonSettings { @@ -153,7 +184,7 @@ object CelebornCommonSettings { val scala213 = "2.13.5" val ALL_SCALA_VERSIONS = Seq(SCALA_2_11_12, SCALA_2_12_10, SCALA_2_12_15, SCALA_2_12_17, SCALA_2_12_18, scala213) - val DEFAULT_SCALA_VERSION = SCALA_2_12_15 + val DEFAULT_SCALA_VERSION = SCALA_2_12_18 val projectScalaVersion = defaultScalaVersion() @@ -258,8 +289,8 @@ object CelebornCommonSettings { pomExtra := https://celeborn.apache.org/ - git@github.com:apache/incubator-celeborn.git - scm:git:git@github.com:apache/incubator-celeborn.git + git@github.com:apache/celeborn.git + scm:git:git@github.com:apache/celeborn.git ) @@ -394,7 +425,10 @@ object CelebornCommon { Dependencies.jacksonDatabind, Dependencies.jacksonAnnotations, Dependencies.log4jSlf4jImpl % "test", - Dependencies.log4j12Api % "test" + Dependencies.log4j12Api % "test", + // SSL support + Dependencies.bouncycastleBcprovJdk18on, + Dependencies.bouncycastleBcpkixJdk18on ) ++ commonUnitTestDependencies, Compile / sourceGenerators += Def.task { @@ -448,21 +482,34 @@ object CelebornService { Dependencies.findbugsJsr305, Dependencies.commonsIo, Dependencies.ioNetty, - Dependencies.javaxServletApi, Dependencies.commonsCrypto, Dependencies.slf4jApi, Dependencies.mybatis, Dependencies.hikaricp, + Dependencies.swaggerJaxrs2, + Dependencies.swaggerUi, + Dependencies.jakartaServletApi, + Dependencies.jerseyServer, + Dependencies.jerseyContainerServletCore, + Dependencies.jerseyHk2, + Dependencies.jerseyMediaJsonJackson, + Dependencies.jerseyMediaMultipart, + Dependencies.jettyServer, + Dependencies.jettyServlet, + Dependencies.jettyProxy, Dependencies.log4jSlf4jImpl % "test", Dependencies.log4j12Api % "test", - Dependencies.h2 % "test" + Dependencies.h2 % "test", + Dependencies.jerseyTestFrameworkCore % "test", + Dependencies.jerseyTestFrameworkProviderJetty % "test" ) ++ commonUnitTestDependencies ) } object CelebornMaster { lazy val master = Project("celeborn-master", file("master")) - .dependsOn(CelebornCommon.common, CelebornService.service) + .dependsOn(CelebornCommon.common) + .dependsOn(CelebornService.service % "test->test;compile->compile") .settings ( commonSettings, protoSettings, @@ -487,6 +534,7 @@ object CelebornWorker { lazy val worker = Project("celeborn-worker", file("worker")) .dependsOn(CelebornService.service) .dependsOn(CelebornCommon.common % "test->test;compile->compile") + .dependsOn(CelebornService.service % "test->test;compile->compile") .dependsOn(CelebornClient.client % "test->compile") .dependsOn(CelebornMaster.master % "test->compile") .settings ( @@ -506,7 +554,9 @@ object CelebornWorker { Dependencies.leveldbJniAll, Dependencies.roaringBitmap, Dependencies.rocksdbJni, - Dependencies.scalatestMockito % "test" + Dependencies.scalatestMockito % "test", + Dependencies.jerseyTestFrameworkCore % "test", + Dependencies.jerseyTestFrameworkProviderJetty % "test" ) ++ commonUnitTestDependencies ) } @@ -714,7 +764,9 @@ trait SparkClientProjects { libraryDependencies ++= Seq( "org.apache.spark" %% "spark-core" % sparkVersion % "test", "org.apache.spark" %% "spark-sql" % sparkVersion % "test", - "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", + "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests" excludeAll( + ExclusionRule("org.glassfish.jersey.inject", "*"), + ExclusionRule("org.glassfish.jersey.core", "*")), "org.apache.spark" %% "spark-sql" % sparkVersion % "test" classifier "tests" ) ++ commonUnitTestDependencies ) diff --git a/service/pom.xml b/service/pom.xml index faf80a142e..b6d0da7399 100644 --- a/service/pom.xml +++ b/service/pom.xml @@ -29,6 +29,36 @@ jar Celeborn Service + + + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + + javax.servlet + javax.servlet-api + + + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-proxy + ${jetty.version} + + + + org.apache.celeborn @@ -43,10 +73,6 @@ io.netty netty-all - - javax.servlet - javax.servlet-api - commons-io commons-io @@ -75,6 +101,62 @@ test + + + io.swagger.core.v3 + swagger-jaxrs2 + + + + org.webjars + swagger-ui + + + + jakarta.servlet + jakarta.servlet-api + + + + org.glassfish.jersey.core + jersey-server + + + + org.glassfish.jersey.containers + jersey-container-servlet-core + + + + org.glassfish.jersey.inject + jersey-hk2 + + + + org.glassfish.jersey.media + jersey-media-json-jackson + + + + org.glassfish.jersey.media + jersey-media-multipart + + + + org.eclipse.jetty + jetty-server + + + + org.eclipse.jetty + jetty-servlet + + + + org.eclipse.jetty + jetty-proxy + + org.mockito @@ -91,5 +173,15 @@ log4j-1.2-api test + + org.glassfish.jersey.test-framework + jersey-test-framework-core + test + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-jetty + test + diff --git a/service/src/main/resources/org/apache/celeborn/swagger/index.html b/service/src/main/resources/org/apache/celeborn/swagger/index.html new file mode 100644 index 0000000000..83e9864438 --- /dev/null +++ b/service/src/main/resources/org/apache/celeborn/swagger/index.html @@ -0,0 +1,73 @@ + + + + + + + Apache Celeborn REST API Documentation + + + + + + +

+ + + + + + diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala b/service/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala similarity index 96% rename from common/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala rename to service/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala index 82311bea96..ab919bb147 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala +++ b/service/src/main/scala/org/apache/celeborn/common/metrics/MetricsSystem.scala @@ -25,11 +25,12 @@ import scala.collection.mutable.ArrayBuffer import scala.util.matching.Regex import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf.{METRICS_JSON_PATH, METRICS_PROMETHEUS_PATH} import org.apache.celeborn.common.internal.Logging -import org.apache.celeborn.common.metrics.sink.{JsonServlet, PrometheusServlet, ServletHttpRequestHandler, Sink} +import org.apache.celeborn.common.metrics.sink.{JsonServlet, PrometheusServlet, Sink} import org.apache.celeborn.common.metrics.source.Source import org.apache.celeborn.common.util.Utils @@ -51,7 +52,7 @@ class MetricsSystem( metricsConfig.initialize() - def getServletHandlers: Array[ServletHttpRequestHandler] = { + def getServletContextHandlers: Array[ServletContextHandler] = { require(running, "Can only call getServletHandlers on a running MetricsSystem") prometheusServlet.map(_.getHandlers(conf)).getOrElse(Array()) ++ jsonServlet.map(_.getHandlers(conf)).getOrElse(Array()) @@ -139,7 +140,7 @@ class MetricsSystem( prometheusServlet = Some(servlet.newInstance( kv._2, registry, - sources.asScala, + sources.asScala.toSeq, prometheusServletPath).asInstanceOf[PrometheusServlet]) } else if (kv._1 == "jsonServlet") { val servlet = Utils.classForName(classPath) @@ -152,7 +153,7 @@ class MetricsSystem( jsonServlet = Some(servlet.newInstance( kv._2, registry, - sources.asScala, + sources.asScala.toSeq, jsonServletPath, conf.metricsJsonPrettyEnabled.asInstanceOf[Object]).asInstanceOf[JsonServlet]) } else { diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala similarity index 77% rename from common/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala rename to service/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala index 3258261647..9b9ba7e123 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala +++ b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/AbstractServlet.scala @@ -16,17 +16,19 @@ */ package org.apache.celeborn.common.metrics.sink +import org.eclipse.jetty.servlet.ServletContextHandler + import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.metrics.source.Source abstract class AbstractServlet(sources: Seq[Source]) extends Sink with Logging { - def getHandlers(conf: CelebornConf): Array[ServletHttpRequestHandler] = { - Array[ServletHttpRequestHandler]( - createHttpRequestHandler()) + def getHandlers(conf: CelebornConf): Array[ServletContextHandler] = { + Array[ServletContextHandler]( + createServletHandler()) } - def createHttpRequestHandler(): ServletHttpRequestHandler + def createServletHandler(): ServletContextHandler def getMetricsSnapshot: String = { sources.map(_.getMetrics).mkString @@ -38,11 +40,3 @@ abstract class AbstractServlet(sources: Seq[Source]) extends Sink with Logging { override def report(): Unit = {} } - -abstract class ServletHttpRequestHandler(path: String) extends Logging { - - def handleRequest(uri: String): String - - def getServletPath(): String = path - -} diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala similarity index 95% rename from common/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala rename to service/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala index 7a2b8b52c8..4ed2c5a3ac 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala +++ b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala @@ -24,10 +24,12 @@ import scala.collection.mutable.ArrayBuffer import com.codahale.metrics.MetricRegistry import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} -import io.netty.channel.ChannelHandler.Sharable +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.celeborn.common.metrics.{CelebornHistogram, CelebornTimer, ResettableSlidingWindowReservoir} -import org.apache.celeborn.common.metrics.source.{AbstractSource, NamedCounter, NamedGauge, NamedHistogram, NamedTimer, Source} +import org.apache.celeborn.common.metrics.source._ +import org.apache.celeborn.server.common.http.HttpUtils +import org.apache.celeborn.server.common.http.HttpUtils.ServletParams object JsonConverter { val mapper = new ObjectMapper() with ClassTagExtensions @@ -72,8 +74,10 @@ class JsonServlet( } } - override def createHttpRequestHandler(): ServletHttpRequestHandler = { - new JsonHttpRequestHandler(servletPath, this) + override def createServletHandler(): ServletContextHandler = { + HttpUtils.createServletHandler( + servletPath, + new ServletParams(_ => getMetricsSnapshot, "text/json")) } override def stop(): Unit = {} @@ -344,12 +348,3 @@ class JsonServlet( } } } - -@Sharable -class JsonHttpRequestHandler(path: String, jsonServlet: JsonServlet) - extends ServletHttpRequestHandler(path) { - - override def handleRequest(uri: String): String = { - jsonServlet.getMetricsSnapshot - } -} diff --git a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala similarity index 73% rename from common/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala rename to service/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala index 4b4548421b..27797c8fc9 100644 --- a/common/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala +++ b/service/src/main/scala/org/apache/celeborn/common/metrics/sink/PrometheusServlet.scala @@ -20,9 +20,11 @@ package org.apache.celeborn.common.metrics.sink import java.util.Properties import com.codahale.metrics.MetricRegistry -import io.netty.channel.ChannelHandler.Sharable +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.celeborn.common.metrics.source.Source +import org.apache.celeborn.server.common.http.HttpUtils +import org.apache.celeborn.server.common.http.HttpUtils.ServletParams class PrometheusServlet( val property: Properties, @@ -30,17 +32,9 @@ class PrometheusServlet( val sources: Seq[Source], val servletPath: String) extends AbstractServlet(sources) { - override def createHttpRequestHandler(): ServletHttpRequestHandler = { - new PrometheusHttpRequestHandler(servletPath, this) - } -} - -@Sharable -class PrometheusHttpRequestHandler( - path: String, - prometheusServlet: PrometheusServlet) extends ServletHttpRequestHandler(path) { - - override def handleRequest(uri: String): String = { - prometheusServlet.getMetricsSnapshot + override def createServletHandler(): ServletContextHandler = { + HttpUtils.createServletHandler( + servletPath, + new ServletParams(_ => getMetricsSnapshot, "text/plain")) } } diff --git a/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala b/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala index f13752b81e..18e016187e 100644 --- a/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala +++ b/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala @@ -23,7 +23,8 @@ import scala.collection.JavaConverters._ import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.internal.Logging -import org.apache.celeborn.server.common.http.{HttpRequestHandler, HttpServer, HttpServerInitializer} +import org.apache.celeborn.server.common.http.HttpServer +import org.apache.celeborn.server.common.http.api.ApiRootResource import org.apache.celeborn.server.common.service.config.ConfigLevel abstract class HttpService extends Service with Logging { @@ -173,18 +174,20 @@ abstract class HttpService extends Service with Logging { def getWorkerEventInfo(): String = throw new UnsupportedOperationException() def startHttpServer(): Unit = { - val handlers = - if (metricsSystem.running) { - new HttpRequestHandler(this, metricsSystem.getServletHandlers) - } else { - new HttpRequestHandler(this, null) - } - httpServer = new HttpServer( + httpServer = HttpServer( serviceName, httpHost(), httpPort(), - new HttpServerInitializer(handlers)) + httpMaxWorkerThreads(), + httpStopTimeout()) httpServer.start() + startInternal() + // block until the HTTP server is started, otherwise, we may get + // the wrong HTTP server port -1 + while (httpServer.getState != "STARTED") { + logInfo(s"Waiting for $serviceName's HTTP server getting started") + Thread.sleep(1000) + } } private def httpHost(): String = { @@ -205,6 +208,41 @@ abstract class HttpService extends Service with Logging { } } + private def httpMaxWorkerThreads(): Int = { + serviceName match { + case Service.MASTER => + conf.masterHttpMaxWorkerThreads + case Service.WORKER => + conf.workerHttpMaxWorkerThreads + } + } + + private def httpStopTimeout(): Long = { + serviceName match { + case Service.MASTER => + conf.masterHttpStopTimeout + case Service.WORKER => + conf.workerHttpStopTimeout + } + } + + def connectionUrl: String = { + httpServer.getServerUri + } + + protected def startInternal(): Unit = { + httpServer.addHandler(ApiRootResource.getServletHandler(this)) + httpServer.addStaticHandler("META-INF/resources/webjars/swagger-ui/4.9.1/", "/swagger-static/") + httpServer.addStaticHandler("org/apache/celeborn/swagger", "/swagger") + httpServer.addRedirectHandler("/help", "/swagger") + httpServer.addRedirectHandler("/docs", "/swagger") + if (metricsSystem.running) { + metricsSystem.getServletContextHandlers.foreach { handler => + httpServer.addHandler(handler) + } + } + } + override def initialize(): Unit = { super.initialize() startHttpServer() diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpEndpoint.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpEndpoint.scala deleted file mode 100644 index 7783558738..0000000000 --- a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpEndpoint.scala +++ /dev/null @@ -1,269 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.celeborn.server.common.http - -import org.apache.celeborn.server.common.{HttpService, Service} -import org.apache.celeborn.server.common.service.config.ConfigLevel - -/** - * HTTP endpoints of Rest API providers. - */ -trait HttpEndpoint { - def path: String - - def description(service: String): String - - def handle(service: HttpService, parameters: Map[String, String]): String -} - -case object Conf extends HttpEndpoint { - override def path: String = "/conf" - - override def description(service: String): String = s"List the conf setting of the $service." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getConf -} - -case object ListDynamicConfigs extends HttpEndpoint { - override def path: String = "/listDynamicConfigs" - - override def description(service: String): String = s"List the dynamic configs of the $service. " + - s"The parameter level specifies the config level of dynamic configs. " + - s"The parameter tenant specifies the tenant id of ${ConfigLevel.TENANT.name()} or ${ConfigLevel.TENANT_USER.name()} level. " + - s"The parameter name specifies the user name of ${ConfigLevel.TENANT_USER.name()} level. " + - s"Meanwhile, either none or all of the parameter tenant and name are specified for ${ConfigLevel.TENANT_USER.name()} level." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getDynamicConfigs( - parameters.getOrElse("LEVEL", "").trim, - parameters.getOrElse("TENANT", "").trim, - parameters.getOrElse("NAME", "").trim) -} - -case object WorkerInfo extends HttpEndpoint { - override def path: String = "/workerInfo" - - override def description(service: String): String = { - if (service == Service.MASTER) - "List worker information of the service. It will list all registered workers 's information." - else "List the worker information of the worker." - } - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getWorkerInfo -} - -case object ThreadDump extends HttpEndpoint { - override def path: String = "/threadDump" - - override def description(service: String): String = - s"List the current thread dump of the $service." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getThreadDump -} - -case object Shuffles extends HttpEndpoint { - override def path: String = "/shuffles" - - override def description(service: String): String = { - if (service == Service.MASTER) - "List all running shuffle keys of the service. It will return all running shuffle's key of the cluster." - else - "List all the running shuffle keys of the worker. It only return keys of shuffles running in that worker." - } - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getShuffleList -} - -case object Applications extends HttpEndpoint { - override def path: String = "/applications" - - override def description(service: String): String = - if (service == Service.MASTER) - "List all running application's ids of the cluster." - else - "List all running application's ids of the worker. It only return application ids running in that worker." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getApplicationList -} - -case object ListTopDiskUsedApps extends HttpEndpoint { - override def path: String = "/listTopDiskUsedApps" - - override def description(service: String): String = { - if (service == Service.MASTER) - "List the top disk usage application ids. It will return the top disk usage application ids for the cluster." - else - "List the top disk usage application ids. It only return application ids running in that worker." - } - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.listTopDiskUseApps -} - -case object Help extends HttpEndpoint { - override def path: String = "/help" - - override def description(service: String): String = - s"List the available API providers of the $service." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - HttpUtils.help(service.serviceName) -} - -case object Invalid extends HttpEndpoint { - - val invalid = "invalid" - - override def path: String = None.toString - - override def description(service: String): String = s"Invalid uri of the $service." - - override def handle(service: HttpService, parameters: Map[String, String]): String = invalid -} - -case object MasterGroupInfo extends HttpEndpoint { - override def path: String = "/masterGroupInfo" - - override def description(service: String): String = - "List master group information of the service. It will list all master's LEADER, FOLLOWER information." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getMasterGroupInfo -} - -case object LostWorkers extends HttpEndpoint { - override def path: String = "/lostWorkers" - - override def description(service: String): String = "List all lost workers of the master." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getLostWorkers -} - -case object ExcludedWorkers extends HttpEndpoint { - override def path: String = "/excludedWorkers" - - override def description(service: String): String = "List all excluded workers of the master." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getExcludedWorkers -} - -case object ShutdownWorkers extends HttpEndpoint { - override def path: String = "/shutdownWorkers" - - override def description(service: String): String = "List all shutdown workers of the master." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getShutdownWorkers -} - -case object Hostnames extends HttpEndpoint { - override def path: String = "/hostnames" - - override def description(service: String): String = - "List all running application's LifecycleManager's hostnames of the cluster." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getHostnameList -} - -case object Exclude extends HttpEndpoint { - override def path: String = "/exclude" - - override def description(service: String): String = - "Excluded workers of the master add or remove the worker manually given worker id. The parameter add or remove specifies the excluded workers to add or remove, which value is separated by commas." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.exclude(parameters.getOrElse("ADD", "").trim, parameters.getOrElse("REMOVE", "").trim) -} - -case object ListPartitionLocationInfo extends HttpEndpoint { - override def path: String = "/listPartitionLocationInfo" - - override def description(service: String): String = - "List all the living PartitionLocation information in that worker." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.listPartitionLocationInfo -} - -case object UnavailablePeers extends HttpEndpoint { - override def path: String = "/unavailablePeers" - - override def description(service: String): String = - "List the unavailable peers of the worker, this always means the worker connect to the peer failed." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getUnavailablePeers -} - -case object IsShutdown extends HttpEndpoint { - override def path: String = "/isShutdown" - - override def description(service: String): String = - "Show if the worker is during the process of shutdown." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.isShutdown -} - -case object IsRegistered extends HttpEndpoint { - override def path: String = "/isRegistered" - - override def description(service: String): String = - "Show if the worker is registered to the master success." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.isRegistered -} - -case object Exit extends HttpEndpoint { - override def path: String = "/exit" - - override def description(service: String): String = - "Trigger this worker to exit. Legal types are 'DECOMMISSION', 'GRACEFUL' and 'IMMEDIATELY'." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.exit(parameters.getOrElse("TYPE", "")) -} - -case object SendWorkerEvent extends HttpEndpoint { - override def path: String = "/sendWorkerEvent" - - override def description(service: String): String = - "For Master(Leader) can send worker event to manager workers. Legal types are 'None', 'Immediately', 'Decommission', 'DecommissionThenIdle', 'Graceful', 'Recommission'" - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.handleWorkerEvent(parameters.getOrElse("TYPE", ""), parameters.getOrElse("WORKERS", "")) -} - -case object WorkerEventInfo extends HttpEndpoint { - override def path: String = "/workerEventInfo" - - override def description(service: String): String = - "List all worker event infos of the master." - - override def handle(service: HttpService, parameters: Map[String, String]): String = - service.getWorkerEventInfo() -} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpRequestHandler.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpRequestHandler.scala deleted file mode 100644 index 531f194547..0000000000 --- a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpRequestHandler.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.celeborn.server.common.http - -import io.netty.buffer.Unpooled -import io.netty.channel.{ChannelFutureListener, ChannelHandlerContext, SimpleChannelInboundHandler} -import io.netty.channel.ChannelHandler.Sharable -import io.netty.handler.codec.http._ -import io.netty.util.CharsetUtil - -import org.apache.celeborn.common.internal.Logging -import org.apache.celeborn.common.metrics.sink.{JsonHttpRequestHandler, ServletHttpRequestHandler} -import org.apache.celeborn.server.common.HttpService - -/** - * A handler for the REST API that defines how to handle the HTTP request given a message. - * - * @param service The service of HTTP server. - * @param uri The uri of HTTP request. - */ -@Sharable -class HttpRequestHandler( - service: HttpService, - servletHttpRequestHandlers: Array[ServletHttpRequestHandler]) - extends SimpleChannelInboundHandler[FullHttpRequest] with Logging { - - override def channelReadComplete(ctx: ChannelHandlerContext): Unit = { - ctx.flush() - } - - override def channelRead0(ctx: ChannelHandlerContext, req: FullHttpRequest): Unit = { - val uri = req.uri() - val (path, parameters) = HttpUtils.parseUri(uri) - val msg = HttpUtils.handleRequest(service, path, parameters) - val textType = "text/plain; charset=UTF-8" - val jsonType = "application/json" - val (response, contentType) = msg match { - case Invalid.invalid => - if (servletHttpRequestHandlers != null) { - servletHttpRequestHandlers.find(servlet => - uri == servlet.getServletPath()).map { - case jsonHandler: JsonHttpRequestHandler => - (jsonHandler.handleRequest(uri), jsonType) - case handler: ServletHttpRequestHandler => - (handler.handleRequest(uri), textType) - }.getOrElse((s"Unknown path $uri!", textType)) - } else { - ( - s"${Invalid.description(service.serviceName)} ${HttpUtils.help(service.serviceName)}", - textType) - } - case _ => (msg, textType) - } - - val res = new DefaultFullHttpResponse( - HttpVersion.HTTP_1_1, - HttpResponseStatus.OK, - Unpooled.copiedBuffer(response, CharsetUtil.UTF_8)) - res.headers().set(HttpHeaderNames.CONTENT_TYPE, contentType) - ctx.writeAndFlush(res).addListener(ChannelFutureListener.CLOSE) - } -} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServer.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServer.scala index a45ff109ab..45d63c6b73 100644 --- a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServer.scala +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpServer.scala @@ -17,75 +17,116 @@ package org.apache.celeborn.server.common.http -import java.net.InetSocketAddress -import java.util.concurrent.TimeUnit - -import io.netty.bootstrap.ServerBootstrap -import io.netty.channel.{ChannelFuture, ChannelInitializer} -import io.netty.channel.socket.nio.NioServerSocketChannel -import io.netty.handler.logging.{LoggingHandler, LogLevel} +import org.apache.commons.lang3.SystemUtils +import org.eclipse.jetty.server.{Handler, HttpConfiguration, HttpConnectionFactory, Server, ServerConnector} +import org.eclipse.jetty.server.handler.{ContextHandlerCollection, ErrorHandler} +import org.eclipse.jetty.util.component.LifeCycle +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.apache.celeborn.common.internal.Logging -import org.apache.celeborn.common.network.util.{IOMode, NettyUtils} -import org.apache.celeborn.common.util.{CelebornExitKind, Utils} +import org.apache.celeborn.common.util.CelebornExitKind -class HttpServer( +private[celeborn] case class HttpServer( role: String, - host: String, - port: Int, - channelInitializer: ChannelInitializer[_]) extends Logging { + server: Server, + connector: ServerConnector, + rootHandler: ContextHandlerCollection) extends Logging { - private var bootstrap: ServerBootstrap = _ - private var bindFuture: ChannelFuture = _ @volatile private var isStarted = false @throws[Exception] def start(): Unit = synchronized { - val boss = NettyUtils.createEventLoop(IOMode.NIO, 1, role + "-http-boss") - val worker = NettyUtils.createEventLoop(IOMode.NIO, 2, role + "-http-worker") - bootstrap = new ServerBootstrap - bootstrap - .group(boss, worker) - .handler(new LoggingHandler(LogLevel.DEBUG)) - .channel(classOf[NioServerSocketChannel]) - .childHandler(channelInitializer) - - val address = new InetSocketAddress(host, port) - bindFuture = bootstrap.bind(address).sync - bindFuture.syncUninterruptibly() - logInfo(s"$role: HttpServer started on ${address.getHostString}:$port.") - isStarted = true + try { + server.start() + connector.start() + server.addConnector(connector) + logInfo(s"$role: HttpServer started on ${connector.getHost}:${connector.getPort}.") + isStarted = true + } catch { + case e: Exception => + stopInternal(CelebornExitKind.EXIT_IMMEDIATELY) + throw e + } } def stop(exitCode: Int): Unit = synchronized { if (isStarted) { - logInfo(s"$role: Stopping HttpServer") - if (bindFuture != null) { - // close is a local operation and should finish within milliseconds; timeout just to be safe - bindFuture.channel.close.awaitUninterruptibly(10, TimeUnit.SECONDS) - bindFuture = null - } - if (bootstrap != null && bootstrap.config.group != null) { - Utils.tryLogNonFatalError { - if (exitCode == CelebornExitKind.WORKER_GRACEFUL_SHUTDOWN) { - bootstrap.config.group.shutdownGracefully(3, 5, TimeUnit.SECONDS) - } else { - bootstrap.config.group.shutdownGracefully(0, 0, TimeUnit.SECONDS) - } - } - } - if (bootstrap != null && bootstrap.config.childGroup != null) { - Utils.tryLogNonFatalError { - if (exitCode == CelebornExitKind.WORKER_GRACEFUL_SHUTDOWN) { - bootstrap.config.childGroup.shutdownGracefully(3, 5, TimeUnit.SECONDS) - } else { - bootstrap.config.childGroup.shutdownGracefully(0, 0, TimeUnit.SECONDS) - } - } - } - bootstrap = null - logInfo(s"$role: HttpServer stopped.") - isStarted = false + stopInternal(exitCode) + } + } + + private def stopInternal(exitCode: Int): Unit = { + if (exitCode == CelebornExitKind.EXIT_IMMEDIATELY) { + server.setStopTimeout(0) + connector.setStopTimeout(0) + } + logInfo(s"$role: Stopping HttpServer") + server.stop() + server.join() + connector.stop() + server.getThreadPool match { + case lifeCycle: LifeCycle => lifeCycle.stop() + case _ => } + logInfo(s"$role: HttpServer stopped.") + isStarted = false + } + + def getServerUri: String = connector.getHost + ":" + connector.getLocalPort + + def addHandler(handler: Handler): Unit = synchronized { + rootHandler.addHandler(handler) + if (!handler.isStarted) handler.start() + } + + def addStaticHandler( + resourceBase: String, + contextPath: String): Unit = { + addHandler(HttpUtils.createStaticHandler(resourceBase, contextPath)) + } + + def addRedirectHandler( + src: String, + dest: String): Unit = { + addHandler(HttpUtils.createRedirectHandler(src, dest)) + } + + def getState: String = server.getState +} + +object HttpServer { + + def apply(role: String, host: String, port: Int, poolSize: Int, stopTimeout: Long): HttpServer = { + val pool = new QueuedThreadPool(math.max(poolSize, 8)) + pool.setName(s"$role-JettyThreadPool") + pool.setDaemon(true) + val server = new Server(pool) + server.setStopTimeout(stopTimeout) + + val errorHandler = new ErrorHandler() + errorHandler.setShowStacks(true) + errorHandler.setServer(server) + server.addBean(errorHandler) + + val collection = new ContextHandlerCollection + server.setHandler(collection) + + val serverExecutor = new ScheduledExecutorScheduler(s"$role-JettyScheduler", true) + val httpConf = new HttpConfiguration() + val connector = new ServerConnector( + server, + null, + serverExecutor, + null, + -1, + -1, + new HttpConnectionFactory(httpConf)) + connector.setHost(host) + connector.setPort(port) + connector.setReuseAddress(!SystemUtils.IS_OS_WINDOWS) + connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8)) + connector.setStopTimeout(stopTimeout) + + new HttpServer(role, server, connector, collection) } } diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpUtils.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpUtils.scala index 4ccb127e5e..b9b6a3ec25 100644 --- a/service/src/main/scala/org/apache/celeborn/server/common/http/HttpUtils.scala +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/HttpUtils.scala @@ -18,78 +18,111 @@ package org.apache.celeborn.server.common.http import java.net.URL -import java.util.Locale - -import org.apache.celeborn.server.common.{HttpService, Service} - -object HttpUtils { - - private val baseEndpoints: List[HttpEndpoint] = - List( - Conf, - ListDynamicConfigs, - WorkerInfo, - ThreadDump, - Shuffles, - Applications, - ListTopDiskUsedApps, - Help) - private val masterEndpoints: List[HttpEndpoint] = List( - MasterGroupInfo, - LostWorkers, - ExcludedWorkers, - ShutdownWorkers, - Hostnames, - SendWorkerEvent, - WorkerEventInfo, - Exclude) ++ baseEndpoints - private val workerEndpoints: List[HttpEndpoint] = - List( - ListPartitionLocationInfo, - UnavailablePeers, - IsShutdown, - IsRegistered, - Exit) ++ baseEndpoints - - def parseUri(uri: String): (String, Map[String, String]) = { - val url = new URL(s"https://127.0.0.1:9000$uri") - val parameter = - if (url.getQuery == null) { - Map.empty[String, String] - } else { - url.getQuery - .split("&") - .map(_.split("=")) - .map(arr => arr(0).toUpperCase(Locale.ROOT) -> arr(1)).toMap +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} + +import org.eclipse.jetty.servlet.{DefaultServlet, ServletContextHandler, ServletHolder} + +import org.apache.celeborn.common.exception.CelebornException +import org.apache.celeborn.common.internal.Logging + +private[celeborn] object HttpUtils extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for + // implicit conversion from many types of functions to jetty Handlers. + type Responder[T] = HttpServletRequest => T + + class ServletParams[T <: AnyRef]( + val responder: Responder[T], + val contentType: String, + val extractFn: T => String = (in: Any) => in.toString) {} + + /** Create a context handler that responds to a request with the given path prefix */ + def createServletHandler[T <: AnyRef]( + path: String, + servletParams: ServletParams[T]): ServletContextHandler = { + createServletHandler(path, createServlet(servletParams)) + } + + private def createServlet[T <: AnyRef](servletParams: ServletParams[T]): HttpServlet = { + new HttpServlet { + override def doGet(request: HttpServletRequest, response: HttpServletResponse): Unit = { + try { + response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) + response.setStatus(HttpServletResponse.SC_OK) + val result = servletParams.responder(request) + response.getWriter.print(servletParams.extractFn(result)) + } catch { + case e: IllegalArgumentException => + response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage) + case e: Exception => + logWarning(s"GET ${request.getRequestURI} failed: $e", e) + throw e + } } - (url.getPath, parameter) + + override protected def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } } - def handleRequest( - service: HttpService, - path: String, - parameters: Map[String, String]): String = { - endpoints(service.serviceName).find(endpoint => endpoint.path == path).orElse( - Some(Invalid)).get.handle( - service, - parameters) + /** + * Create a handler for serving files from a static directory + * + * @param resourceBase the resource directory contains static resource files + * @param contextPath the content path to set for the handler + * @return a static [[ServletContextHandler]] + */ + def createStaticHandler( + resourceBase: String, + contextPath: String): ServletContextHandler = { + val contextHandler = new ServletContextHandler() + val holder = new ServletHolder(classOf[DefaultServlet]) + Option(Thread.currentThread().getContextClassLoader.getResource(resourceBase)) match { + case Some(res) => + holder.setInitParameter("resourceBase", res.toString) + case None => + throw new CelebornException("Could not find resource path for Web UI: " + resourceBase) + } + contextHandler.setContextPath(contextPath) + contextHandler.addServlet(holder, "/") + contextHandler } - def help(service: String): String = { - val sb = new StringBuilder - sb.append("Available API providers include:\n") - val httpEndpoints: List[HttpEndpoint] = endpoints(service) - val maxLength = httpEndpoints.map(_.path.length).max - httpEndpoints.sortBy(_.path).foreach(endpoint => - sb.append( - s"${endpoint.path.padTo(maxLength, " ").mkString} ${endpoint.description(service)}\n")) - sb.toString + def createServletHandler(contextPath: String, servlet: HttpServlet): ServletContextHandler = { + val handler = new ServletContextHandler() + val holder = new ServletHolder(servlet) + handler.setContextPath(contextPath) + handler.addServlet(holder, "/") + handler } - private def endpoints(service: String): List[HttpEndpoint] = { - if (service == Service.MASTER) - masterEndpoints - else - workerEndpoints + def createRedirectHandler(src: String, dest: String): ServletContextHandler = { + val redirectedServlet = new HttpServlet { + private def doReq(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + val newURL = new URL(new URL(req.getRequestURL.toString), dest).toString + resp.sendRedirect(newURL) + } + override def doGet(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + doReq(req, resp) + } + + override def doPut(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + doReq(req, resp) + } + + override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + doReq(req, resp) + } + + override def doDelete(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + doReq(req, resp) + } + + override protected def doTrace(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + + createServletHandler(src, redirectedServlet) } } diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala new file mode 100644 index 0000000000..45a5d27968 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import javax.ws.rs.{GET, Path, Produces, QueryParam} +import javax.ws.rs.core.MediaType + +import io.swagger.v3.oas.annotations.media.Content +import io.swagger.v3.oas.annotations.responses.ApiResponse + +@Path("/") +private[api] class ApiBaseResource extends ApiRequestContext { + def service: String = httpService.serviceName + + @GET + @Path("ping") + @Produces(Array(MediaType.TEXT_PLAIN)) + def ping(): String = "pong" + + @Path("/conf") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List the conf setting.") + @GET + def conf: String = httpService.getConf + + @Path("/listDynamicConfigs") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List the dynamic configs. " + + "The parameter level specifies the config level of dynamic configs. " + + "The parameter tenant specifies the tenant id of TENANT or TENANT_USER level. " + + "The parameter name specifies the user name of TENANT_USER level. " + + "Meanwhile, either none or all of the parameter tenant and name are specified for TENANT_USER level.") + @GET + def listDynamicConfigs( + @QueryParam("LEVEL") level: String, + @QueryParam("TENANT") tenant: String, + @QueryParam("NAME") name: String): String = { + httpService.getDynamicConfigs( + normalizeParam(level), + normalizeParam(tenant), + normalizeParam(name)) + } + + @Path("/workerInfo") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "For MASTER: List worker information of the service. It will list all registered workers 's information.\n" + + "For WORKER: List the worker information of the worker.") + @GET + def workerInfo(): String = { + httpService.getWorkerInfo + } + + @Path("/threadDump") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List the current thread dump.") + @GET + def threadDump(): String = { + httpService.getThreadDump + } + + @Path("shuffle") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "For MASTER: List all running shuffle keys of the service. It will return all running shuffle's key of the cluster.\n" + + "For WORKER: List all the running shuffle keys of the worker. It only return keys of shuffles running in that worker.") + @GET + def shuffles(): String = { + httpService.getShuffleList + } + + @Path("applications") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "For MASTER: List all running application's ids of the cluster.\n" + + "For WORKER: List all running application's ids of the worker. It only return application ids running in that worker.") + @GET + def applications(): String = { + httpService.getApplicationList + } + + @Path("listTopDiskUsedApps") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "For MASTER: List the top disk usage application ids. It will return the top disk usage application ids for the cluster.\n" + + "For WORKER: List the top disk usage application ids. It only return application ids running in that worker.") + @GET + def listTopDiskUsedApps(): String = { + httpService.listTopDiskUseApps + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRequestContext.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRequestContext.scala new file mode 100644 index 0000000000..2c293bf29d --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRequestContext.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import javax.servlet.ServletContext +import javax.servlet.http.HttpServletRequest +import javax.ws.rs.core.Context + +import org.eclipse.jetty.server.handler.ContextHandler + +import org.apache.celeborn.server.common.HttpService + +private[celeborn] trait ApiRequestContext { + @Context + protected var servletContext: ServletContext = _ + + @Context + protected var httpRequest: HttpServletRequest = _ + + final protected def httpService: HttpService = HttpServiceContext.get(servletContext) + + protected def normalizeParam(param: String): String = Option(param).map(_.trim).getOrElse("") +} + +private[celeborn] object HttpServiceContext { + private val attribute = getClass.getCanonicalName + + def set(contextHandler: ContextHandler, rs: HttpService): Unit = { + contextHandler.setAttribute(attribute, rs) + } + + def get(context: ServletContext): HttpService = { + context.getAttribute(attribute).asInstanceOf[HttpService] + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRootResource.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRootResource.scala new file mode 100644 index 0000000000..6d69c1b58a --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiRootResource.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ResourceConfig +import org.glassfish.jersey.servlet.ServletContainer + +import org.apache.celeborn.server.common.HttpService + +private[celeborn] object ApiRootResource { + def getServletHandler(rs: HttpService): ServletContextHandler = { + val openapiConf: ResourceConfig = new OpenAPIConfig + val holder = new ServletHolder(new ServletContainer(openapiConf)) + val handler = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) + handler.setContextPath("/") + HttpServiceContext.set(handler, rs) + handler.addServlet(holder, "/*") + handler + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornOpenApiResource.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornOpenApiResource.scala new file mode 100644 index 0000000000..5559ed90d2 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornOpenApiResource.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import javax.servlet.ServletConfig +import javax.ws.rs.{GET, Path, PathParam, Produces} +import javax.ws.rs.core.{Application, Context, HttpHeaders, MediaType, Response, UriInfo} + +import scala.collection.JavaConverters._ + +import com.fasterxml.jackson.core.util.DefaultPrettyPrinter +import io.swagger.v3.jaxrs2.integration.JaxrsOpenApiContextBuilder +import io.swagger.v3.jaxrs2.integration.resources.BaseOpenApiResource +import io.swagger.v3.oas.annotations.Operation +import io.swagger.v3.oas.integration.api.OpenApiContext +import io.swagger.v3.oas.models.OpenAPI +import io.swagger.v3.oas.models.info.{Info, License} +import io.swagger.v3.oas.models.servers.Server +import org.apache.commons.lang3.StringUtils + +@Path("/openapi.{type:json|yaml}") +class CelebornOpenApiResource extends BaseOpenApiResource with ApiRequestContext { + @Context + protected var config: ServletConfig = _ + + @Context + protected var app: Application = _ + + @GET + @Produces(Array(MediaType.APPLICATION_JSON, "application/yaml")) + @Operation(hidden = true) + def getOpenApi( + @Context headers: HttpHeaders, + @Context uriInfo: UriInfo, + @PathParam("type") tpe: String): Response = { + + val ctxId = getContextId(config) + val ctx: OpenApiContext = new CelebornJaxrsOpenApiContextBuilder() + .servletConfig(config) + .application(app) + .resourcePackages(OpenAPIConfig.packages.toSet.asJava) + .configLocation(configLocation) + .openApiConfiguration(openApiConfiguration) + .ctxId(ctxId) + .buildContext(true) + + val openApi = setCelebornOpenAPIDefinition(ctx.read()) + + if (StringUtils.isNotBlank(tpe) && tpe.trim().equalsIgnoreCase("yaml")) { + Response.status(Response.Status.OK) + .entity( + ctx.getOutputYamlMapper() + .writer(new DefaultPrettyPrinter()) + .writeValueAsString(openApi)) + .`type`("application/yaml") + .build() + } else { + Response.status(Response.Status.OK) + .entity( + ctx.getOutputJsonMapper + .writer(new DefaultPrettyPrinter()) + .writeValueAsString(openApi)) + .`type`(MediaType.APPLICATION_JSON_TYPE) + .build() + } + } + + private def setCelebornOpenAPIDefinition(openApi: OpenAPI): OpenAPI = { + // TODO: to improve when https is enabled. + val apiUrl = s"http://${httpService.connectionUrl}/" + openApi.info( + new Info().title( + s"Apache Celeborn REST API Documentation") + .description(s"Role: ${httpService.serviceName}") + .license( + new License().name("Apache License 2.0") + .url("https://www.apache.org/licenses/LICENSE-2.0.txt"))) + .servers(List(new Server().url(apiUrl)).asJava) + } +} + +class CelebornJaxrsOpenApiContextBuilder + extends JaxrsOpenApiContextBuilder[CelebornJaxrsOpenApiContextBuilder] diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornScalaObjectMapper.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornScalaObjectMapper.scala new file mode 100644 index 0000000000..37375e2c19 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/CelebornScalaObjectMapper.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import javax.ws.rs.ext.ContextResolver + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +class CelebornScalaObjectMapper extends ContextResolver[ObjectMapper] { + private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + + override def getContext(aClass: Class[_]): ObjectMapper = mapper +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/http/api/OpenAPIConfig.scala b/service/src/main/scala/org/apache/celeborn/server/common/http/api/OpenAPIConfig.scala new file mode 100644 index 0000000000..07053de4ce --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/http/api/OpenAPIConfig.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http.api + +import org.glassfish.jersey.server.ResourceConfig + +class OpenAPIConfig extends ResourceConfig { + packages(OpenAPIConfig.packages: _*) + register(classOf[CelebornOpenApiResource]) + register(classOf[CelebornScalaObjectMapper]) +} + +object OpenAPIConfig { + val packages = Seq( + "org.apache.celeborn.server.common.http.api", + "org.apache.celeborn.service.deploy.master.http.api", + "org.apache.celeborn.service.deploy.worker.http.api") +} diff --git a/service/src/test/resources/metrics-api.properties b/service/src/test/resources/metrics-api.properties new file mode 100644 index 0000000000..b1b2e67254 --- /dev/null +++ b/service/src/test/resources/metrics-api.properties @@ -0,0 +1,18 @@ +# +# 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. +# +*.sink.prometheusServlet.class=org.apache.celeborn.common.metrics.sink.PrometheusServlet +*.sink.jsonServlet.class=org.apache.celeborn.common.metrics.sink.JsonServlet diff --git a/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala b/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala new file mode 100644 index 0000000000..632d06c79c --- /dev/null +++ b/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http + +import javax.ws.rs.core.MediaType + +import org.apache.celeborn.common.CelebornConf + +abstract class ApiBaseResourceSuite extends HttpTestHelper { + celebornConf.set(CelebornConf.METRICS_ENABLED.key, "true") + .set( + CelebornConf.METRICS_CONF.key, + Thread.currentThread().getContextClassLoader.getResource("metrics-api.properties").getFile) + + test("ping") { + val response = webTarget.path("ping").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + assert(response.readEntity(classOf[String]) == "pong") + } + + test("conf") { + val response = webTarget.path("conf").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("listDynamicConfigs") { + val response = webTarget.path("listDynamicConfigs") + .queryParam("LEVEL", "TENANT") + .request(MediaType.TEXT_PLAIN) + .get() + assert(200 == response.getStatus) + } + + test("workerInfo") { + val response = webTarget.path("workerInfo").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("threadDump") { + val response = webTarget.path("threadDump").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("shuffle") { + val response = webTarget.path("shuffle").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("applications") { + val response = webTarget.path("applications").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("listTopDiskUsedApps") { + val response = webTarget.path("listTopDiskUsedApps").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("openapi.json") { + val response = webTarget.path("openapi.json").request(MediaType.APPLICATION_JSON).get() + assert(200 == response.getStatus) + assert(response.readEntity(classOf[String]).contains("/conf")) + } + + test("swagger") { + Seq("swagger", "docs", "help").foreach { path => + val response = webTarget.path(path).request(MediaType.TEXT_HTML).get() + assert(200 == response.getStatus) + assert(response.readEntity(classOf[String]).contains("swagger-ui")) + } + } + + test("metrics/prometheus") { + val response = webTarget.path("metrics/prometheus").request(MediaType.APPLICATION_JSON).get() + assert(200 == response.getStatus) + assert(response.readEntity(classOf[String]).contains("metrics_jvm_memory_heap_max_Value")) + } + + test("metrics/json") { + val response = webTarget.path("metrics/json").request(MediaType.APPLICATION_JSON).get() + assert(200 == response.getStatus) + assert(response.readEntity(classOf[String]).contains("\"name\" : \"jvm.memory.heap.max\"")) + } +} diff --git a/service/src/test/scala/org/apache/celeborn/server/common/http/HttpTestHelper.scala b/service/src/test/scala/org/apache/celeborn/server/common/http/HttpTestHelper.scala new file mode 100644 index 0000000000..8674fc39cd --- /dev/null +++ b/service/src/test/scala/org/apache/celeborn/server/common/http/HttpTestHelper.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.server.common.http + +import java.net.URI +import javax.ws.rs.client.WebTarget +import javax.ws.rs.core.{Application, UriBuilder} + +import org.glassfish.jersey.client.ClientConfig +import org.glassfish.jersey.media.multipart.MultiPartFeature +import org.glassfish.jersey.server.ResourceConfig +import org.glassfish.jersey.test.JerseyTest +import org.glassfish.jersey.test.jetty.JettyTestContainerFactory +import org.glassfish.jersey.test.spi.TestContainerFactory +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.server.common.HttpService +import org.apache.celeborn.server.common.http.HttpTestHelper.RestApiBaseSuite +import org.apache.celeborn.server.common.http.api.CelebornScalaObjectMapper + +object HttpTestHelper { + class RestApiBaseSuite extends JerseyTest { + + override def configure: Application = new ResourceConfig(getClass) + .register(classOf[MultiPartFeature]) + + override def configureClient(config: ClientConfig): Unit = { + config.register(classOf[CelebornScalaObjectMapper]) + .register(classOf[MultiPartFeature]) + } + + override def getTestContainerFactory: TestContainerFactory = new JettyTestContainerFactory + } +} + +trait HttpTestHelper extends AnyFunSuite + with BeforeAndAfterAll + with BeforeAndAfterEach + with Logging { + + protected val celebornConf = new CelebornConf() + protected def httpService: HttpService + + protected val restApiBaseSuite: JerseyTest = new RestApiBaseSuite + + override def beforeAll(): Unit = { + super.beforeAll() + restApiBaseSuite.setUp() + } + + override def afterAll(): Unit = { + restApiBaseSuite.tearDown() + super.afterAll() + } + + protected lazy val baseUri: URI = + UriBuilder.fromUri(s"http://${httpService.connectionUrl}/").build() + + protected lazy val webTarget: WebTarget = restApiBaseSuite.client.target(baseUri) +} diff --git a/service/src/test/scala/org/apache/celeborn/server/common/http/HttpUtilsSuite.scala b/service/src/test/scala/org/apache/celeborn/server/common/http/HttpUtilsSuite.scala deleted file mode 100644 index cbb5d18efa..0000000000 --- a/service/src/test/scala/org/apache/celeborn/server/common/http/HttpUtilsSuite.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.celeborn.server.common.http - -import org.scalatest.funsuite.AnyFunSuite - -import org.apache.celeborn.common.internal.Logging -import org.apache.celeborn.server.common.Service - -class HttpUtilsSuite extends AnyFunSuite with Logging { - - def checkParseUri( - uri: String, - expectPath: String, - expectParameters: Map[String, String]): Unit = { - val (path, parameters) = HttpUtils.parseUri(uri) - assert(path == expectPath) - assert(parameters == expectParameters) - } - - test("CELEBORN-448: Support exclude worker manually") { - checkParseUri("/exclude", "/exclude", Map.empty) - checkParseUri( - "/exclude?add=localhost:1001:1002:1003:1004", - "/exclude", - Map("ADD" -> "localhost:1001:1002:1003:1004")) - checkParseUri( - "/exclude?remove=localhost:1001:1002:1003:1004", - "/exclude", - Map("REMOVE" -> "localhost:1001:1002:1003:1004")) - checkParseUri( - "/exclude?add=localhost:1001:1002:1003:1004&remove=localhost:2001:2002:2003:2004", - "/exclude", - Map("ADD" -> "localhost:1001:1002:1003:1004", "REMOVE" -> "localhost:2001:2002:2003:2004")) - } - - test("CELEBORN-847: Support parse HTTP Restful API parameters") { - checkParseUri("/exit", "/exit", Map.empty) - checkParseUri("/exit?type=decommission", "/exit", Map("TYPE" -> "decommission")) - checkParseUri( - "/exit?type=decommission&foo=A", - "/exit", - Map("TYPE" -> "decommission", "FOO" -> "A")) - } - - test("CELEBORN-829: Improve response message of invalid HTTP request") { - assert(HttpUtils.help(Service.MASTER) == - s"""Available API providers include: - |/applications List all running application's ids of the cluster. - |/conf List the conf setting of the master. - |/exclude Excluded workers of the master add or remove the worker manually given worker id. The parameter add or remove specifies the excluded workers to add or remove, which value is separated by commas. - |/excludedWorkers List all excluded workers of the master. - |/help List the available API providers of the master. - |/hostnames List all running application's LifecycleManager's hostnames of the cluster. - |/listDynamicConfigs List the dynamic configs of the master. The parameter level specifies the config level of dynamic configs. The parameter tenant specifies the tenant id of TENANT or TENANT_USER level. The parameter name specifies the user name of TENANT_USER level. Meanwhile, either none or all of the parameter tenant and name are specified for TENANT_USER level. - |/listTopDiskUsedApps List the top disk usage application ids. It will return the top disk usage application ids for the cluster. - |/lostWorkers List all lost workers of the master. - |/masterGroupInfo List master group information of the service. It will list all master's LEADER, FOLLOWER information. - |/sendWorkerEvent For Master(Leader) can send worker event to manager workers. Legal types are 'None', 'Immediately', 'Decommission', 'DecommissionThenIdle', 'Graceful', 'Recommission' - |/shuffles List all running shuffle keys of the service. It will return all running shuffle's key of the cluster. - |/shutdownWorkers List all shutdown workers of the master. - |/threadDump List the current thread dump of the master. - |/workerEventInfo List all worker event infos of the master. - |/workerInfo List worker information of the service. It will list all registered workers 's information. - |""".stripMargin) - assert(HttpUtils.help(Service.WORKER) == - s"""Available API providers include: - |/applications List all running application's ids of the worker. It only return application ids running in that worker. - |/conf List the conf setting of the worker. - |/exit Trigger this worker to exit. Legal types are 'DECOMMISSION', 'GRACEFUL' and 'IMMEDIATELY'. - |/help List the available API providers of the worker. - |/isRegistered Show if the worker is registered to the master success. - |/isShutdown Show if the worker is during the process of shutdown. - |/listDynamicConfigs List the dynamic configs of the worker. The parameter level specifies the config level of dynamic configs. The parameter tenant specifies the tenant id of TENANT or TENANT_USER level. The parameter name specifies the user name of TENANT_USER level. Meanwhile, either none or all of the parameter tenant and name are specified for TENANT_USER level. - |/listPartitionLocationInfo List all the living PartitionLocation information in that worker. - |/listTopDiskUsedApps List the top disk usage application ids. It only return application ids running in that worker. - |/shuffles List all the running shuffle keys of the worker. It only return keys of shuffles running in that worker. - |/threadDump List the current thread dump of the worker. - |/unavailablePeers List the unavailable peers of the worker, this always means the worker connect to the peer failed. - |/workerInfo List the worker information of the worker. - |""".stripMargin) - } - - test("CELEBORN-1245: Support Master manage workers") { - checkParseUri( - "/sendWorkerEvent?type=decommission&workers=localhost:1001:1002:1003:1004", - "/sendWorkerEvent", - Map("TYPE" -> "decommission", "WORKERS" -> "localhost:1001:1002:1003:1004")) - } - - test("CELEBORN-1056: Introduce Rest API of listing dynamic configuration") { - checkParseUri("/listDynamicConfigs", "/listDynamicConfigs", Map.empty) - checkParseUri( - "/listDynamicConfigs?level=system", - "/listDynamicConfigs", - Map("LEVEL" -> "system")) - checkParseUri( - "/listDynamicConfigs?level=tenant&tenant=tenantId1", - "/listDynamicConfigs", - Map("LEVEL" -> "tenant", "TENANT" -> "tenantId1")) - checkParseUri( - "/listDynamicConfigs?level=tenant_user&tenant=tenantId1&name=user1", - "/listDynamicConfigs", - Map("LEVEL" -> "tenant_user", "TENANT" -> "tenantId1", "NAME" -> "user1")) - } -} diff --git a/tests/spark-it/pom.xml b/tests/spark-it/pom.xml index 75e50221b2..842d7a8ed9 100644 --- a/tests/spark-it/pom.xml +++ b/tests/spark-it/pom.xml @@ -75,6 +75,14 @@ log4j log4j + + org.glassfish.jersey.inject + * + + + org.glassfish.jersey.core + * + diff --git a/web/README.md b/web/README.md index eae6e43284..c11506c70b 100644 --- a/web/README.md +++ b/web/README.md @@ -4,7 +4,7 @@ Celeborn Web is a dashboard to display and manage the Master and Worker of Celeb > **โš ๏ธ Important** > -> Before running commands, you must ensure that you are in the front-end directory `incubator-celeborn/web`. If not, run `cd web` first. +> Before running commands, you must ensure that you are in the front-end directory `celeborn/web`. If not, run `cd web` first. --- diff --git a/worker/pom.xml b/worker/pom.xml index c4a7c913d9..3821fe9dae 100644 --- a/worker/pom.xml +++ b/worker/pom.xml @@ -98,6 +98,12 @@ ${project.version} test + + org.apache.celeborn + celeborn-service_${scala.binary.version} + ${project.version} + test-jar + org.apache.celeborn celeborn-master_${scala.binary.version} @@ -109,6 +115,16 @@ mockito-scala-scalatest_${scala.binary.version} test + + org.glassfish.jersey.test-framework + jersey-test-framework-core + test + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-jetty + test + diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala index 6bd86908cd..7eedb1d01c 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala @@ -148,7 +148,7 @@ class FetchHandler( val endIndices = openStreamList.getEndIndexList val readLocalFlags = openStreamList.getReadLocalShuffleList val pbOpenStreamListResponse = PbOpenStreamListResponse.newBuilder() - + checkAuth(client, Utils.splitShuffleKey(shuffleKey)._1) 0 until files.size() foreach { idx => val pbStreamHandlerOpt = handleReduceOpenStreamInternal( client, @@ -341,6 +341,7 @@ class FetchHandler( isLegacy: Boolean, readLocalShuffle: Boolean = false, callback: RpcResponseCallback): Unit = { + checkAuth(client, Utils.splitShuffleKey(shuffleKey)._1) workerSource.recordAppActiveConnection(client, shuffleKey) workerSource.startTimer(WorkerSource.OPEN_STREAM_TIME, shuffleKey) try { diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala index 7988ad287b..400256b0fa 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala @@ -119,6 +119,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler client, pushData, pushData.requestId, + pushData.shuffleKey, () => { val partitionType = shufflePartitionType.getOrDefault(pushData.shuffleKey, PartitionType.REDUCE) @@ -143,6 +144,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler client, pushMergedData, pushMergedData.requestId, + pushMergedData.shuffleKey, () => handlePushMergedData( pushMergedData, @@ -748,8 +750,10 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler client: TransportClient, message: RequestMessage, requestId: Long, + shuffleKey: String, handler: () => Unit, callback: RpcResponseCallback): Unit = { + checkAuth(client, Utils.splitShuffleKey(shuffleKey)._1) try { handler() } catch { @@ -843,6 +847,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler client, rpcRequest, requestId, + shuffleKey, () => handleMapPartitionRpcRequestCore( requestId, diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResource.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResource.scala new file mode 100644 index 0000000000..8573b71520 --- /dev/null +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResource.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.service.deploy.worker.http.api + +import javax.ws.rs.{GET, Path, POST, QueryParam} +import javax.ws.rs.core.MediaType + +import io.swagger.v3.oas.annotations.media.Content +import io.swagger.v3.oas.annotations.responses.ApiResponse + +import org.apache.celeborn.server.common.http.api.ApiRequestContext + +@Path("/") +class ApiWorkerResource extends ApiRequestContext { + @Path("/listPartitionLocationInfo") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "List all the living PartitionLocation information in that worker.") + @GET + def listPartitionLocationInfo: String = httpService.listPartitionLocationInfo + + @Path("/unavailablePeers") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "List the unavailable peers of the worker, this always means the worker connect to the peer failed.") + @GET + def unavailablePeers: String = httpService.getUnavailablePeers + + @Path("/isShutdown") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "Show if the worker is during the process of shutdown.") + @GET + def isShutdown: String = httpService.isShutdown + + @Path("/isRegistered") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = "Show if the worker is registered to the master success.") + @GET + def isRegistered: String = httpService.isRegistered + + @Path("/exit") + @ApiResponse( + responseCode = "200", + content = Array(new Content( + mediaType = MediaType.TEXT_PLAIN)), + description = + "Trigger this worker to exit. Legal types are 'DECOMMISSION', 'GRACEFUL' and 'IMMEDIATELY'.") + @POST + def exit(@QueryParam("TYPE") exitType: String): String = { + httpService.exit(normalizeParam(exitType)) + } +} diff --git a/worker/src/test/java/org/apache/celeborn/service/deploy/worker/FetchHandlerSuiteJ.java b/worker/src/test/java/org/apache/celeborn/service/deploy/worker/FetchHandlerSuiteJ.java index c473efc78f..ed8b40353d 100644 --- a/worker/src/test/java/org/apache/celeborn/service/deploy/worker/FetchHandlerSuiteJ.java +++ b/worker/src/test/java/org/apache/celeborn/service/deploy/worker/FetchHandlerSuiteJ.java @@ -322,7 +322,7 @@ private FetchHandler mockFetchHandler(FileInfo fileInfo) { return fetchHandler; } - private final String shuffleKey = "dummyShuffleKey"; + private final String shuffleKey = "dummyShuffleKey-123"; private final String fileName = "dummyFileName"; private final long dummyRequestId = 0; diff --git a/worker/src/test/java/org/apache/celeborn/service/deploy/worker/storage/local/DiskReducePartitionDataWriterSuiteJ.java b/worker/src/test/java/org/apache/celeborn/service/deploy/worker/storage/local/DiskReducePartitionDataWriterSuiteJ.java index 04c68cbb7a..cd26d8515e 100644 --- a/worker/src/test/java/org/apache/celeborn/service/deploy/worker/storage/local/DiskReducePartitionDataWriterSuiteJ.java +++ b/worker/src/test/java/org/apache/celeborn/service/deploy/worker/storage/local/DiskReducePartitionDataWriterSuiteJ.java @@ -201,7 +201,7 @@ public ByteBuffer createOpenMessage() { new TransportMessage( MessageType.OPEN_STREAM, PbOpenStream.newBuilder() - .setShuffleKey("shuffleKey") + .setShuffleKey("shuffleKey-123") .setFileName("location") .setStartIndex(0) .setEndIndex(Integer.MAX_VALUE) diff --git a/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala b/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala index beecb76a58..914512cca0 100644 --- a/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala +++ b/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala @@ -17,6 +17,7 @@ package org.apache.celeborn.service.deploy +import java.io.IOException import java.net.BindException import java.nio.file.Files import java.util.concurrent.locks.{Lock, ReentrantLock} @@ -70,7 +71,9 @@ trait MiniClusterFeature extends Logging { workers = w created = true } catch { - case e: BindException => + case e: IOException + if e.isInstanceOf[BindException] || Option(e.getCause).exists( + _.isInstanceOf[BindException]) => logError(s"failed to setup mini cluster, retrying (retry count: $retryCount)", e) retryCount += 1 if (retryCount == 3) { diff --git a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResourceSuite.scala b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResourceSuite.scala new file mode 100644 index 0000000000..d950d06742 --- /dev/null +++ b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/ApiWorkerResourceSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.celeborn.service.deploy.worker.http.api + +import javax.ws.rs.core.MediaType + +import org.apache.celeborn.server.common.HttpService +import org.apache.celeborn.server.common.http.ApiBaseResourceSuite +import org.apache.celeborn.service.deploy.MiniClusterFeature +import org.apache.celeborn.service.deploy.worker.Worker + +class ApiWorkerResourceSuite extends ApiBaseResourceSuite with MiniClusterFeature { + private var worker: Worker = _ + override protected def httpService: HttpService = worker + + override def beforeAll(): Unit = { + logInfo("test initialized, setup celeborn mini cluster") + val (_, w) = + setupMiniClusterWithRandomPorts(workerConf = celebornConf.getAll.toMap, workerNum = 1) + worker = w.head + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + logInfo("all test complete, stop celeborn mini cluster") + shutdownMiniCluster() + } + + test("listPartitionLocationInfo") { + val response = webTarget.path("listPartitionLocationInfo").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("unavailablePeers") { + val response = webTarget.path("unavailablePeers").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("isShutdown") { + val response = webTarget.path("isShutdown").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } + + test("isRegistered") { + val response = webTarget.path("isRegistered").request(MediaType.TEXT_PLAIN).get() + assert(200 == response.getStatus) + } +}