Skip to content

Commit bb1cfde

Browse files
ulysses-youyaooqinn
authored andcommitted
[KYUUBI #1800] Remove oom hook
<!-- 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. --> closes #1800 ### _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 - [ ] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1803 from ulysses-you/KYUUBI#1800. Closes #1800 416bf9f [ulysses-you] comment 9843769 [ulysses-you] style git diff ad9d21d [ulysses-you] remove oom hook Authored-by: ulysses-you <ulyssesyou18@gmail.com> Signed-off-by: Kent Yao <yao@apache.org>
1 parent dd4c2fa commit bb1cfde

File tree

6 files changed

+13
-200
lines changed

6 files changed

+13
-200
lines changed

kyuubi-common/src/main/scala/org/apache/kyuubi/service/TBinaryFrontendService.scala

Lines changed: 7 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.kyuubi.service
1919

20-
import java.util.concurrent.TimeUnit
20+
import java.util.concurrent.{SynchronousQueue, ThreadPoolExecutor, TimeUnit}
2121

2222
import org.apache.hive.service.rpc.thrift._
2323
import org.apache.thrift.protocol.TBinaryProtocol
@@ -26,7 +26,7 @@ import org.apache.thrift.transport.TServerSocket
2626

2727
import org.apache.kyuubi.{KyuubiException, Logging}
2828
import org.apache.kyuubi.config.KyuubiConf
29-
import org.apache.kyuubi.util.ExecutorPoolCaptureOom
29+
import org.apache.kyuubi.util.NamedThreadFactory
3030

3131
/**
3232
* Apache Thrift based hive service rpc
@@ -46,27 +46,21 @@ abstract class TBinaryFrontendService(name: String)
4646

4747
private var server: Option[TServer] = None
4848

49-
// When a OOM occurs, here we de-register the engine by stop its discoveryService.
50-
// Then the current engine will not be connected by new client anymore but keep the existing ones
51-
// alive. In this case we can reduce the engine's overhead and make it possible recover from that.
52-
// We shall not tear down the whole engine by serverable.stop to make the engine unreachable for
53-
// the existing clients which are still getting statuses and reporting to the end-users.
54-
protected def oomHook: Runnable = {
55-
() => discoveryService.foreach(_.stop())
56-
}
49+
// Removed OOM hook since Kyuubi #1800 to respect the hive server2 #2383
5750

5851
override def initialize(conf: KyuubiConf): Unit = synchronized {
5952
this.conf = conf
6053
try {
6154
val minThreads = conf.get(FRONTEND_THRIFT_MIN_WORKER_THREADS)
6255
val maxThreads = conf.get(FRONTEND_THRIFT_MAX_WORKER_THREADS)
6356
val keepAliveTime = conf.get(FRONTEND_THRIFT_WORKER_KEEPALIVE_TIME)
64-
val executor = ExecutorPoolCaptureOom(
65-
name + "Handler-Pool",
57+
val executor = new ThreadPoolExecutor(
6658
minThreads,
6759
maxThreads,
6860
keepAliveTime,
69-
oomHook)
61+
TimeUnit.MILLISECONDS,
62+
new SynchronousQueue[Runnable](),
63+
new NamedThreadFactory(name + "Handler-Pool", false))
7064
val transFactory = authFactory.getTTransportFactory
7165
val tProcFactory = authFactory.getTProcessorFactory(this)
7266
val tServerSocket = new TServerSocket(serverSocket)

kyuubi-common/src/main/scala/org/apache/kyuubi/util/ExecutorPoolCaptureOom.scala

Lines changed: 0 additions & 53 deletions
This file was deleted.

kyuubi-common/src/test/scala/org/apache/kyuubi/service/NoopTBinaryFrontendService.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,4 @@ class NoopTBinaryFrontendService(override val serverable: Serverable)
2121
extends TBinaryFrontendService("NoopThriftBinaryFrontend") {
2222

2323
override val discoveryService: Option[Service] = None
24-
25-
override protected def oomHook: Runnable = () => serverable.stop()
2624
}

kyuubi-common/src/test/scala/org/apache/kyuubi/util/ExecutorPoolCaptureOomSuite.scala

Lines changed: 0 additions & 121 deletions
This file was deleted.

kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiMySQLFrontendService.scala

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
package org.apache.kyuubi.server
1919

2020
import java.net.{InetAddress, InetSocketAddress}
21-
import java.util.concurrent.{ThreadPoolExecutor, TimeUnit}
21+
import java.util.concurrent.{SynchronousQueue, ThreadPoolExecutor, TimeUnit}
2222

2323
import io.netty.bootstrap.ServerBootstrap
2424
import io.netty.buffer.PooledByteBufAllocator
@@ -32,7 +32,7 @@ import org.apache.kyuubi.config.KyuubiConf._
3232
import org.apache.kyuubi.server.mysql._
3333
import org.apache.kyuubi.server.mysql.authentication.MySQLAuthHandler
3434
import org.apache.kyuubi.service.{AbstractFrontendService, Serverable, Service}
35-
import org.apache.kyuubi.util.ExecutorPoolCaptureOom
35+
import org.apache.kyuubi.util.NamedThreadFactory
3636
import org.apache.kyuubi.util.NettyUtils._
3737

3838
/**
@@ -50,18 +50,17 @@ class KyuubiMySQLFrontendService(override val serverable: Serverable)
5050

5151
@volatile protected var isStarted = false
5252

53-
protected def oomHook: Runnable = () => serverable.stop()
54-
5553
override def initialize(conf: KyuubiConf): Unit = synchronized {
5654
val minThreads = conf.get(FRONTEND_MYSQL_MIN_WORKER_THREADS)
5755
val maxThreads = conf.get(FRONTEND_MYSQL_MAX_WORKER_THREADS)
5856
val keepAliveMs = conf.get(FRONTEND_MYSQL_WORKER_KEEPALIVE_TIME)
59-
execPool = ExecutorPoolCaptureOom(
60-
"mysql-exec-pool",
57+
execPool = new ThreadPoolExecutor(
6158
minThreads,
6259
maxThreads,
6360
keepAliveMs,
64-
oomHook)
61+
TimeUnit.MILLISECONDS,
62+
new SynchronousQueue[Runnable](),
63+
new NamedThreadFactory("mysql-exec-pool", false))
6564

6665
serverAddr = conf.get(FRONTEND_MYSQL_BIND_HOST)
6766
.map(InetAddress.getByName)

kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTBinaryFrontendService.scala

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -69,9 +69,5 @@ final class KyuubiTBinaryFrontendService(
6969
resp
7070
}
7171

72-
override protected def oomHook: Runnable = {
73-
() => serverable.stop()
74-
}
75-
7672
override protected def isServer(): Boolean = true
7773
}

0 commit comments

Comments
 (0)