Skip to content

Commit

Permalink
[SPARK-43187][TEST] Remove workaround for MiniKdc's BindException
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR basically reverts the SPARK-31631, which was aimed to address [HADOOP-12656](https://issues.apache.org/jira/browse/HADOOP-12656)

### Why are the changes needed?

Since [HADOOP-12656](https://issues.apache.org/jira/browse/HADOOP-12656) got fixed in Hadoop 2.8.0/3.0.0, and SPARK-42452 removed support for Hadoop2, we can remove this workaround now.

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

No.

### How was this patch tested?

Pass GA.

Closes apache#40849 from pan3793/SPARK-43187.

Authored-by: Cheng Pan <[email protected]>
Signed-off-by: Chao Sun <[email protected]>
  • Loading branch information
pan3793 authored and sunchao committed Apr 19, 2023
1 parent 2097581 commit 77b72fc
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 53 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import javax.security.auth.login.Configuration

import scala.collection.JavaConverters._
import scala.io.Source
import scala.util.control.NonFatal

import com.google.common.io.Files
import kafka.api.Request
Expand Down Expand Up @@ -139,30 +138,8 @@ class KafkaTestUtils(
val kdcDir = Utils.createTempDir()
val kdcConf = MiniKdc.createConf()
kdcConf.setProperty(MiniKdc.DEBUG, "true")
// The port for MiniKdc service gets selected in the constructor, but will be bound
// to it later in MiniKdc.start() -> MiniKdc.initKDCServer() -> KdcServer.start().
// In meantime, when some other service might capture the port during this progress, and
// cause BindException.
// This makes our tests which have dedicated JVMs and rely on MiniKDC being flaky
//
// https://issues.apache.org/jira/browse/HADOOP-12656 get fixed in Hadoop 2.8.0.
//
// The workaround here is to periodically repeat this process with a timeout , since we are
// using Hadoop 2.7.4 as default.
// https://issues.apache.org/jira/browse/SPARK-31631
eventually(timeout(60.seconds), interval(1.second)) {
try {
kdc = new MiniKdc(kdcConf, kdcDir)
kdc.start()
} catch {
case NonFatal(e) =>
if (kdc != null) {
kdc.stop()
kdc = null
}
throw e
}
}
kdc = new MiniKdc(kdcConf, kdcDir)
kdc.start()
// TODO https://issues.apache.org/jira/browse/SPARK-30037
// Need to build spark's own MiniKDC and customize krb5.conf like Kafka
rewriteKrb5Conf()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,10 @@ package org.apache.spark.deploy.security

import java.security.PrivilegedExceptionAction

import scala.util.control.NonFatal

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION
import org.apache.hadoop.minikdc.MiniKdc
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.deploy.SparkHadoopUtil
Expand Down Expand Up @@ -92,30 +88,8 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite {
// krb5.conf. MiniKdc sets "java.security.krb5.conf" in start and removes it when stop called.
val kdcDir = Utils.createTempDir()
val kdcConf = MiniKdc.createConf()
// The port for MiniKdc service gets selected in the constructor, but will be bound
// to it later in MiniKdc.start() -> MiniKdc.initKDCServer() -> KdcServer.start().
// In meantime, when some other service might capture the port during this progress, and
// cause BindException.
// This makes our tests which have dedicated JVMs and rely on MiniKDC being flaky
//
// https://issues.apache.org/jira/browse/HADOOP-12656 get fixed in Hadoop 2.8.0.
//
// The workaround here is to periodically repeat this process with a timeout , since we are
// using Hadoop 2.7.4 as default.
// https://issues.apache.org/jira/browse/SPARK-31631
eventually(timeout(60.seconds), interval(1.second)) {
try {
kdc = new MiniKdc(kdcConf, kdcDir)
kdc.start()
} catch {
case NonFatal(e) =>
if (kdc != null) {
kdc.stop()
kdc = null
}
throw e
}
}
kdc = new MiniKdc(kdcConf, kdcDir)
kdc.start()

val krbConf = new Configuration()
krbConf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos")
Expand Down

0 comments on commit 77b72fc

Please sign in to comment.