Skip to content

Commit ff0441c

Browse files
hddongyaooqinn
authored andcommitted
[KYUUBI #1446] Decouple zookeeper from abstract ServiceDiscovery
<!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html 2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'. 3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'. --> ### _Why are the changes needed?_ <!-- Please clarify why the changes are needed. For instance, 1. If you add a feature, you can talk about the use case of it. 2. If you fix a bug, you can clarify why it is a bug. --> Decouple zookeeper from abstract ServiceDiscovery ### _How was this patch tested?_ - [X] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [X] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1448 from hddong/decouple-zeekeeper. Closes #1446 0294c3c [hongdongdong] remove 5da2f94 [hongdongdong] fix be066f6 [hongdongdong] fix 0303855 [hongdongdong] [KYUUBI #1446] Decouple zookeeper from abstract ServiceDiscovery Authored-by: hongdongdong <hongdongdong@cmss.chinamobile.com> Signed-off-by: Kent Yao <yao@apache.org>
1 parent 10034cd commit ff0441c

File tree

3 files changed

+266
-188
lines changed

3 files changed

+266
-188
lines changed

kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/EngineServiceDiscovery.scala

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -31,13 +31,14 @@ class EngineServiceDiscovery(
3131
fe: FrontendService) extends ServiceDiscovery("EngineServiceDiscovery", fe) {
3232

3333
override def stop(): Unit = synchronized {
34-
closeServiceNode()
34+
discoveryClient.deregisterService()
3535
conf.get(ENGINE_SHARE_LEVEL) match {
3636
// For connection level, we should clean up the namespace in zk in case the disk stress.
37-
case "CONNECTION" if namespace != null =>
37+
case "CONNECTION" =>
3838
try {
39-
zkClient.delete().deletingChildrenIfNeeded().forPath(namespace)
40-
info("Clean up discovery service due to this is connection share level.")
39+
if (discoveryClient.postDeregisterService) {
40+
info("Clean up discovery service due to this is connection share level.")
41+
}
4142
} catch {
4243
case NonFatal(e) =>
4344
warn("Failed to clean up Spark engine before stop.", e)

kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ServiceDiscovery.scala

Lines changed: 11 additions & 184 deletions
Original file line numberDiff line numberDiff line change
@@ -17,28 +17,20 @@
1717

1818
package org.apache.kyuubi.ha.client
1919

20-
import java.io.IOException
2120
import java.nio.charset.StandardCharsets
22-
import java.util.concurrent.TimeUnit
23-
import java.util.concurrent.atomic.AtomicBoolean
2421

2522
import scala.collection.JavaConverters._
2623

2724
import com.google.common.annotations.VisibleForTesting
2825
import org.apache.curator.framework.CuratorFramework
29-
import org.apache.curator.framework.recipes.nodes.PersistentNode
30-
import org.apache.curator.framework.state.{ConnectionState, ConnectionStateListener}
31-
import org.apache.curator.framework.state.ConnectionState.{CONNECTED, LOST, RECONNECTED}
3226
import org.apache.curator.utils.ZKPaths
33-
import org.apache.zookeeper.{CreateMode, KeeperException, WatchedEvent, Watcher}
34-
import org.apache.zookeeper.CreateMode.PERSISTENT
35-
import org.apache.zookeeper.KeeperException.NodeExistsException
3627

37-
import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiException, Logging}
28+
import org.apache.kyuubi.Logging
3829
import org.apache.kyuubi.config.KyuubiConf
3930
import org.apache.kyuubi.ha.HighAvailabilityConf._
31+
import org.apache.kyuubi.ha.client.zookeeper.ServiceDiscoveryClient
32+
import org.apache.kyuubi.ha.client.zookeeper.ServiceDiscoveryClient.createServiceNode
4033
import org.apache.kyuubi.service.{AbstractService, FrontendService}
41-
import org.apache.kyuubi.util.{KyuubiHadoopUtils, ThreadUtils}
4234

4335
/**
4436
* A abstract service for service discovery
@@ -48,92 +40,31 @@ import org.apache.kyuubi.util.{KyuubiHadoopUtils, ThreadUtils}
4840
*/
4941
abstract class ServiceDiscovery(
5042
name: String,
51-
fe: FrontendService) extends AbstractService(name) {
43+
val fe: FrontendService) extends AbstractService(name) {
5244

53-
import ServiceDiscovery._
54-
import ZooKeeperClientProvider._
45+
private var _discoveryClient: ServiceDiscoveryClient = _
5546

56-
private var _zkClient: CuratorFramework = _
57-
private var _serviceNode: PersistentNode = _
58-
59-
/**
60-
* a pre-defined namespace used to publish the instance of the associate service
61-
*/
62-
private var _namespace: String = _
63-
64-
def zkClient: CuratorFramework = _zkClient
65-
66-
def serviceNode: PersistentNode = _serviceNode
67-
68-
def namespace: String = _namespace
47+
def discoveryClient: ServiceDiscoveryClient = _discoveryClient
6948

7049
override def initialize(conf: KyuubiConf): Unit = {
7150
this.conf = conf
72-
_namespace = conf.get(HA_ZK_NAMESPACE)
73-
val maxSleepTime = conf.get(HA_ZK_CONN_MAX_RETRY_WAIT)
74-
val maxRetries = conf.get(HA_ZK_CONN_MAX_RETRIES)
75-
_zkClient = buildZookeeperClient(conf)
76-
zkClient.getConnectionStateListenable.addListener(new ConnectionStateListener {
77-
private val isConnected = new AtomicBoolean(false)
7851

79-
override def stateChanged(client: CuratorFramework, newState: ConnectionState): Unit = {
80-
info(s"Zookeeper client connection state changed to: $newState")
81-
newState match {
82-
case CONNECTED | RECONNECTED => isConnected.set(true)
83-
case LOST =>
84-
isConnected.set(false)
85-
val delay = maxRetries.toLong * maxSleepTime
86-
connectionChecker.schedule(
87-
new Runnable {
88-
override def run(): Unit = if (!isConnected.get()) {
89-
error(s"Zookeeper client connection state changed to: $newState, but failed to" +
90-
s" reconnect in ${delay / 1000} seconds. Give up retry. ")
91-
stopGracefully()
92-
}
93-
},
94-
delay,
95-
TimeUnit.MILLISECONDS)
96-
case _ =>
97-
}
98-
}
99-
})
100-
zkClient.start()
52+
_discoveryClient = new ServiceDiscoveryClient(this)
53+
discoveryClient.createClient(conf)
54+
10155
super.initialize(conf)
10256
}
10357

10458
override def start(): Unit = {
105-
val instance = fe.connectionUrl
106-
_serviceNode = createServiceNode(conf, zkClient, namespace, instance)
107-
// Set a watch on the serviceNode
108-
val watcher = new DeRegisterWatcher
109-
if (zkClient.checkExists.usingWatcher(watcher).forPath(serviceNode.getActualPath) == null) {
110-
// No node exists, throw exception
111-
throw new KyuubiException(s"Unable to create znode for this Kyuubi " +
112-
s"instance[${fe.connectionUrl}] on ZooKeeper.")
113-
}
59+
discoveryClient.registerService(conf)
11460
super.start()
11561
}
11662

11763
override def stop(): Unit = {
118-
closeServiceNode()
119-
if (zkClient != null) zkClient.close()
64+
discoveryClient.closeClient()
12065
super.stop()
12166
}
12267

123-
// close the EPHEMERAL_SEQUENTIAL node in zk
124-
protected def closeServiceNode(): Unit = {
125-
if (_serviceNode != null) {
126-
try {
127-
_serviceNode.close()
128-
} catch {
129-
case e: IOException =>
130-
error("Failed to close the persistent ephemeral znode" + serviceNode.getActualPath, e)
131-
} finally {
132-
_serviceNode = null
133-
}
134-
}
135-
}
136-
13768
// stop the server genteelly
13869
def stopGracefully(): Unit = {
13970
stop()
@@ -143,23 +74,10 @@ abstract class ServiceDiscovery(
14374
fe.serverable.stop()
14475
}
14576

146-
class DeRegisterWatcher extends Watcher {
147-
override def process(event: WatchedEvent): Unit = {
148-
if (event.getType == Watcher.Event.EventType.NodeDeleted) {
149-
warn(s"This Kyuubi instance ${fe.connectionUrl} is now de-registered from" +
150-
s" ZooKeeper. The server will be shut down after the last client session completes.")
151-
stopGracefully()
152-
}
153-
}
154-
}
155-
15677
}
15778

15879
object ServiceDiscovery extends Logging {
15980

160-
final private lazy val connectionChecker =
161-
ThreadUtils.newDaemonSingleThreadScheduledExecutor("zk-connection-checker")
162-
16381
def supportServiceDiscovery(conf: KyuubiConf): Boolean = {
16482
val zkEnsemble = conf.get(HA_ZK_QUORUM)
16583
zkEnsemble != null && zkEnsemble.nonEmpty
@@ -234,97 +152,6 @@ object ServiceDiscovery extends Logging {
234152
external: Boolean = false): String = {
235153
createServiceNode(conf, zkClient, namespace, instance, version, external).getActualPath
236154
}
237-
238-
private def createServiceNode(
239-
conf: KyuubiConf,
240-
zkClient: CuratorFramework,
241-
namespace: String,
242-
instance: String,
243-
version: Option[String] = None,
244-
external: Boolean = false): PersistentNode = {
245-
val ns = ZKPaths.makePath(null, namespace)
246-
try {
247-
zkClient
248-
.create()
249-
.creatingParentsIfNeeded()
250-
.withMode(PERSISTENT)
251-
.forPath(ns)
252-
} catch {
253-
case _: NodeExistsException => // do nothing
254-
case e: KeeperException =>
255-
throw new KyuubiException(s"Failed to create namespace '$ns'", e)
256-
}
257-
258-
val session = conf.get(HA_ZK_ENGINE_REF_ID)
259-
.map(refId => s"refId=$refId;").getOrElse("")
260-
val pathPrefix = ZKPaths.makePath(
261-
namespace,
262-
s"serviceUri=$instance;version=${version.getOrElse(KYUUBI_VERSION)};${session}sequence=")
263-
var serviceNode: PersistentNode = null
264-
val createMode =
265-
if (external) CreateMode.PERSISTENT_SEQUENTIAL
266-
else CreateMode.EPHEMERAL_SEQUENTIAL
267-
val znodeData =
268-
if (conf.get(HA_ZK_PUBLIST_CONFIGS) && session.isEmpty) {
269-
addConfsToPublish(conf, instance)
270-
} else {
271-
instance
272-
}
273-
try {
274-
serviceNode = new PersistentNode(
275-
zkClient,
276-
createMode,
277-
false,
278-
pathPrefix,
279-
znodeData.getBytes(StandardCharsets.UTF_8))
280-
serviceNode.start()
281-
val znodeTimeout = conf.get(HA_ZK_NODE_TIMEOUT)
282-
if (!serviceNode.waitForInitialCreate(znodeTimeout, TimeUnit.MILLISECONDS)) {
283-
throw new KyuubiException(s"Max znode creation wait time $znodeTimeout s exhausted")
284-
}
285-
info(s"Created a ${serviceNode.getActualPath} on ZooKeeper for KyuubiServer uri: " + instance)
286-
} catch {
287-
case e: Exception =>
288-
if (serviceNode != null) {
289-
serviceNode.close()
290-
}
291-
throw new KyuubiException(
292-
s"Unable to create a znode for this server instance: $instance",
293-
e)
294-
}
295-
serviceNode
296-
}
297-
298-
/**
299-
* Refer to the implementation of HIVE-11581 to simplify user connection parameters.
300-
* https://issues.apache.org/jira/browse/HIVE-11581
301-
* HiveServer2 should store connection params in ZK
302-
* when using dynamic service discovery for simpler client connection string.
303-
*/
304-
private def addConfsToPublish(conf: KyuubiConf, instance: String): String = {
305-
if (!instance.contains(":")) {
306-
return instance
307-
}
308-
val hostPort = instance.split(":", 2)
309-
val confsToPublish = collection.mutable.Map[String, String]()
310-
311-
// Hostname
312-
confsToPublish += ("hive.server2.thrift.bind.host" -> hostPort(0))
313-
// Transport mode
314-
confsToPublish += ("hive.server2.transport.mode" -> "binary")
315-
// Transport specific confs
316-
confsToPublish += ("hive.server2.thrift.port" -> hostPort(1))
317-
confsToPublish += ("hive.server2.thrift.sasl.qop" -> conf.get(KyuubiConf.SASL_QOP))
318-
// Auth specific confs
319-
val authenticationMethod = conf.get(KyuubiConf.AUTHENTICATION_METHOD).mkString(",")
320-
confsToPublish += ("hive.server2.authentication" -> authenticationMethod)
321-
if (authenticationMethod.equalsIgnoreCase("KERBEROS")) {
322-
confsToPublish += ("hive.server2.authentication.kerberos.principal" ->
323-
conf.get(KyuubiConf.SERVER_PRINCIPAL).map(KyuubiHadoopUtils.getServerPrincipal)
324-
.getOrElse(""))
325-
}
326-
confsToPublish.map { case (k, v) => k + "=" + v }.mkString(";")
327-
}
328155
}
329156

330157
case class ServiceNodeInfo(

0 commit comments

Comments
 (0)