[CELEBORN-1711][TEST] Fix '65535' port is invalid

### What changes were proposed in this pull request?

Fix flaky test caused by invalid port.
```
[info] ApiMasterResourceSuite:
[info] org.apache.celeborn.service.deploy.master.http.api.ApiMasterResourceSuite *** ABORTED ***
[info]   java.lang.IllegalArgumentException: '65535' in celeborn.master.http.port is invalid. Invalid port
```

### Why are the changes needed?

The ports range in CelebornConf is [1024, 65535), 65535 is excluded.

169b6f6973/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala (L2315-L2324)

<img width="928" alt="image" src="https://github.com/user-attachments/assets/4532b1bc-c548-45cd-b836-c493f2904422">

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?

GA.

Closes #2901 from turboFei/fix_invalid_port.

Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
This commit is contained in:
Wang, Fei 2024-11-11 23:39:41 -08:00
parent be4c02e6d0
commit aa625496ee
15 changed files with 28 additions and 21 deletions

View File

@ -58,7 +58,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -58,7 +58,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -65,7 +65,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -65,7 +65,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -65,7 +65,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -65,7 +65,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -68,7 +68,7 @@ public class RemoteShuffleMasterSuiteJ {
@Before
public void setUp() {
configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
remoteShuffleMaster = createShuffleMaster(configuration);

View File

@ -53,7 +53,7 @@ public class CelebornTierMasterAgentSuiteJ {
@Before
public void setUp() {
Configuration configuration = new Configuration();
int startPort = Utils$.MODULE$.selectRandomPort(1024, 65535);
int startPort = Utils$.MODULE$.selectRandomInt(1024, 65535);
configuration.setInteger("celeborn.master.port", startPort);
configuration.setString("celeborn.master.endpoints", "localhost:" + startPort);
masterAgent = createMasterAgent(configuration);

View File

@ -245,8 +245,15 @@ object Utils extends Logging {
}
}
def selectRandomPort(from: Int, to: Int): Int = {
ScalaRandom.nextInt(to - from) + from
/**
* Select a random integer within the specified range.
*
* @param from the lower bound of the range (inclusive)
* @param until the upper bound of the range (exclusive)
* @return a randomly selected integer within the range [from, until)
*/
def selectRandomInt(from: Int, until: Int): Int = {
ScalaRandom.nextInt(until - 1 - from) + from
}
def startServiceOnPort[T](

View File

@ -119,7 +119,7 @@ public class RatisMasterStatusSystemSuiteJ {
String id2 = UUID.randomUUID().toString();
String id3 = UUID.randomUUID().toString();
int ratisPort1 = Utils$.MODULE$.selectRandomPort(1024, 65535);
int ratisPort1 = Utils$.MODULE$.selectRandomInt(1024, 65535);
int ratisPort2 = ratisPort1 + 1;
int ratisPort3 = ratisPort2 + 1;

View File

@ -36,7 +36,7 @@ class ApiMasterResourceAuthenticationSuite extends ApiBaseResourceAuthentication
}
override def beforeAll(): Unit = {
val randomMasterPort = Utils.selectRandomPort(1024, 65535)
val randomMasterPort = Utils.selectRandomInt(1024, 65535)
val randomHttpPort = randomMasterPort + 1
celebornConf.set(CelebornConf.HA_ENABLED.key, "false")
celebornConf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir())

View File

@ -41,7 +41,7 @@ class MasterSuite extends AnyFunSuite
test("test single node startup functionality") {
val conf = new CelebornConf()
val randomMasterPort = Utils.selectRandomPort(1024, 65535)
val randomMasterPort = Utils.selectRandomInt(1024, 65535)
val randomHttpPort = randomMasterPort + 1
conf.set(CelebornConf.HA_ENABLED.key, "false")
conf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir())
@ -104,7 +104,7 @@ class MasterSuite extends AnyFunSuite
test("test master worker host allow and deny pattern") {
val conf = new CelebornConf()
val randomMasterPort = Utils.selectRandomPort(1024, 65535)
val randomMasterPort = Utils.selectRandomInt(1024, 65535)
val randomHttpPort = randomMasterPort + 1
conf.set(CelebornConf.HA_ENABLED.key, "false")
conf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir())

View File

@ -39,7 +39,7 @@ class ApiMasterResourceSuite extends ApiBaseResourceSuite {
}
override def beforeAll(): Unit = {
val randomMasterPort = Utils.selectRandomPort(1024, 65535)
val randomMasterPort = Utils.selectRandomInt(1024, 65535)
val randomHttpPort = randomMasterPort + 1
celebornConf.set(CelebornConf.HA_ENABLED.key, "false")
celebornConf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir())

View File

@ -42,7 +42,7 @@ class ApiV1MasterResourceSuite extends ApiV1BaseResourceSuite {
}
override def beforeAll(): Unit = {
val randomMasterPort = Utils.selectRandomPort(1024, 65535)
val randomMasterPort = Utils.selectRandomInt(1024, 65535)
val randomHttpPort = randomMasterPort + 1
celebornConf.set(CelebornConf.HA_ENABLED.key, "false")
celebornConf.set(CelebornConf.HA_MASTER_RATIS_STORAGE_DIR.key, getTmpDir())

View File

@ -28,7 +28,7 @@ import scala.collection.mutable
import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.util.{CelebornExitKind, Utils}
import org.apache.celeborn.common.util.Utils.selectRandomPort
import org.apache.celeborn.common.util.Utils.selectRandomInt
import org.apache.celeborn.service.deploy.master.{Master, MasterArguments}
import org.apache.celeborn.service.deploy.worker.{Worker, WorkerArguments}
import org.apache.celeborn.service.deploy.worker.memory.MemoryManager
@ -56,8 +56,8 @@ trait MiniClusterFeature extends Logging {
var workers: collection.Set[Worker] = null
while (!created) {
try {
val randomPort = selectRandomPort(1024, 65535)
val randomInternalPort = selectRandomPort(1024, 65535)
val randomPort = selectRandomInt(1024, 65535)
val randomInternalPort = selectRandomInt(1024, 65535)
val finalMasterConf = Map(
s"${CelebornConf.MASTER_HOST.key}" -> "localhost",
s"${CelebornConf.PORT_MAX_RETRY.key}" -> "0",
@ -103,7 +103,7 @@ trait MiniClusterFeature extends Logging {
private def createMaster(map: Map[String, String] = null): Master = {
val conf = new CelebornConf()
conf.set(CelebornConf.METRICS_ENABLED.key, "false")
val httpPort = selectRandomPort(1024, 65535)
val httpPort = selectRandomInt(1024, 65535)
conf.set(CelebornConf.MASTER_HTTP_PORT.key, s"$httpPort")
logInfo(s"set ${CelebornConf.MASTER_HTTP_PORT.key} to $httpPort")
if (map != null) {
@ -128,7 +128,7 @@ trait MiniClusterFeature extends Logging {
conf.set(CelebornConf.WORKER_STORAGE_DIRS.key, storageDir)
conf.set(CelebornConf.WORKER_DISK_MONITOR_ENABLED.key, "false")
conf.set(CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE.key, "256K")
conf.set(CelebornConf.WORKER_HTTP_PORT.key, s"${selectRandomPort(1024, 65535)}")
conf.set(CelebornConf.WORKER_HTTP_PORT.key, s"${selectRandomInt(1024, 65535)}")
conf.set("celeborn.fetch.io.threads", "4")
conf.set("celeborn.push.io.threads", "4")
if (map != null) {