You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ya...@apache.org on 2021/09/03 11:19:27 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #999] Build the basic framework for rest frontend service

This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 32111a3  [KYUUBI #999] Build the basic framework for rest frontend service
32111a3 is described below

commit 32111a3014e52e222bcbb1dbdf8f9dae4f9e0f9c
Author: yanghua <ya...@gmail.com>
AuthorDate: Fri Sep 3 19:19:17 2021 +0800

    [KYUUBI #999] Build the basic framework for rest frontend service
    
    <!--
    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.
    -->
    
    ### _How was this patch tested?_
    - [ ] 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 #1003 from yanghua/KYUUBI-999.
    
    Closes #999
    
    78fdf542 [yanghua] Changed rest frontend service port from 10009 to 10099
    73929d5d [yanghua] [KYUUBI #999] Build the basic framework for rest frontend service
    
    Authored-by: yanghua <ya...@gmail.com>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 dev/dependencyList                                 |  16 +++
 docs/deployment/settings.md                        |   2 +
 .../org/apache/kyuubi/config/KyuubiConf.scala      |  15 +++
 .../org/apache/kyuubi/service/NoopServer.scala     |   3 +-
 kyuubi-server/pom.xml                              |  15 +++
 .../apache/kyuubi/server/RestFrontendService.scala | 127 +++++++++++++++++++++
 .../scala/org/apache/kyuubi/server/api/api.scala   |  75 ++++++++++++
 .../kyuubi/server/api/v1/ApiRootResource.scala     |  33 ++++++
 .../kyuubi/server/RestFrontendServiceSuite.scala   |  89 +++++++++++++++
 pom.xml                                            |  26 +++++
 10 files changed, 400 insertions(+), 1 deletion(-)

diff --git a/dev/dependencyList b/dev/dependencyList
index b28e2ad..6c13f62 100644
--- a/dev/dependencyList
+++ b/dev/dependencyList
@@ -15,6 +15,7 @@
 # limitations under the License.
 #
 
+aopalliance-repackaged/2.5.0//aopalliance-repackaged-2.5.0.jar
 commons-codec/1.15//commons-codec-1.15.jar
 commons-lang3/3.10//commons-lang3-3.10.jar
 curator-client/2.12.0//curator-client-2.12.0.jar
@@ -25,15 +26,28 @@ guava/30.1-jre//guava-30.1-jre.jar
 hadoop-client-api/3.2.2//hadoop-client-api-3.2.2.jar
 hadoop-client-runtime/3.2.2//hadoop-client-runtime-3.2.2.jar
 hive-service-rpc/2.3.7//hive-service-rpc-2.3.7.jar
+hk2-api/2.5.0//hk2-api-2.5.0.jar
+hk2-locator/2.5.0//hk2-locator-2.5.0.jar
+hk2-utils/2.5.0//hk2-utils-2.5.0.jar
 htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar
 jackson-annotations/2.11.4//jackson-annotations-2.11.4.jar
 jackson-core/2.11.4//jackson-core-2.11.4.jar
 jackson-databind/2.11.4//jackson-databind-2.11.4.jar
 jackson-module-paranamer/2.11.4//jackson-module-paranamer-2.11.4.jar
 jackson-module-scala_2.12/2.11.4//jackson-module-scala_2.12-2.11.4.jar
+jakarta.annotation-api/1.3.4//jakarta.annotation-api-1.3.4.jar
+jakarta.inject/2.5.0//jakarta.inject-2.5.0.jar
 jakarta.servlet-api/4.0.4//jakarta.servlet-api-4.0.4.jar
+jakarta.ws.rs-api/2.1.5//jakarta.ws.rs-api-2.1.5.jar
+javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar
 jaxb-api/2.2.11//jaxb-api-2.2.11.jar
 jcl-over-slf4j/1.7.30//jcl-over-slf4j-1.7.30.jar
+jersey-client/2.29//jersey-client-2.29.jar
+jersey-common/2.29//jersey-common-2.29.jar
+jersey-container-servlet-core/2.29//jersey-container-servlet-core-2.29.jar
+jersey-hk2/2.29//jersey-hk2-2.29.jar
+jersey-media-jaxb/2.29//jersey-media-jaxb-2.29.jar
+jersey-server/2.29//jersey-server-2.29.jar
 jetty-http/9.4.41.v20210516//jetty-http-9.4.41.v20210516.jar
 jetty-io/9.4.41.v20210516//jetty-io-9.4.41.v20210516.jar
 jetty-security/9.4.41.v20210516//jetty-security-9.4.41.v20210516.jar
@@ -48,6 +62,7 @@ metrics-core/4.1.1//metrics-core-4.1.1.jar
 metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar
 metrics-json/4.1.1//metrics-json-4.1.1.jar
 metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar
+osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar
 paranamer/2.8//paranamer-2.8.jar
 scala-library/2.12.14//scala-library-2.12.14.jar
 scopt_2.12/4.0.1//scopt_2.12-4.0.1.jar
@@ -57,4 +72,5 @@ simpleclient_dropwizard/0.10.0//simpleclient_dropwizard-0.10.0.jar
 simpleclient_servlet/0.10.0//simpleclient_servlet-0.10.0.jar
 slf4j-api/1.7.30//slf4j-api-1.7.30.jar
 slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar
+validation-api/2.0.1.Final//validation-api-2.0.1.Final.jar
 zookeeper/3.4.14//zookeeper-3.4.14.jar
diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index edc121b..c62b7ac 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -198,6 +198,8 @@ kyuubi\.frontend\.login<br>\.timeout|<div style='width: 65pt;word-wrap: break-wo
 kyuubi\.frontend\.max<br>\.message\.size|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>104857600</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>(deprecated) Maximum message size in bytes a Kyuubi server will accept.</div>|<div style='width: 30pt'>int</div>|<div style='width: 20pt'>1.0.0</div>
 kyuubi\.frontend\.max<br>\.worker\.threads|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>999</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>(deprecated) Maximum number of threads in the of frontend worker thread pool for the thrift frontend service</div>|<div style='width: 30pt'>int</div>|<div style='width: 20pt'>1.0.0</div>
 kyuubi\.frontend\.min<br>\.worker\.threads|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>9</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>(deprecated) Minimum number of threads in the of frontend worker thread pool for the thrift frontend service</div>|<div style='width: 30pt'>int</div>|<div style='width: 20pt'>1.0.0</div>
+kyuubi\.frontend\.rest<br>\.bind\.host|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>&lt;undefined&gt;</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>Hostname or IP of the machine on which to run the REST frontend service.</div>|<div style='width: 30pt'>string</div>|<div style='width: 20pt'>1.4.0</div>
+kyuubi\.frontend\.rest<br>\.bind\.port|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>10099</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>Port of the machine on which to run the REST frontend service.</div>|<div style='width: 30pt'>int</div>|<div style='width: 20pt'>1.4.0</div>
 kyuubi\.frontend<br>\.thrift\.backoff\.slot<br>\.length|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>PT0.1S</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>Time to back off during login to the thrift frontend service.</div>|<div style='width: 30pt'>duration</div>|<div style='width: 20pt'>1.4.0</div>
 kyuubi\.frontend<br>\.thrift\.binary\.bind<br>\.host|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>&lt;undefined&gt;</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>Hostname or IP of the machine on which to run the thrift frontend service via binary protocol.</div>|<div style='width: 30pt'>string</div>|<div style='width: 20pt'>1.4.0</div>
 kyuubi\.frontend<br>\.thrift\.binary\.bind<br>\.port|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>10009</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>Port of the machine on which to run the thrift frontend service via binary protocol.</div>|<div style='width: 30pt'>int</div>|<div style='width: 20pt'>1.4.0</div>
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index ad7b17e..03e55e1 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -134,6 +134,8 @@ case class KyuubiConf(loadSysDefault: Boolean = true) extends Logging {
     FRONTEND_BIND_PORT,
     FRONTEND_THRIFT_BINARY_BIND_HOST,
     FRONTEND_THRIFT_BINARY_BIND_PORT,
+    FRONTEND_REST_BIND_HOST,
+    FRONTEND_REST_BIND_PORT,
     AUTHENTICATION_METHOD,
     SERVER_KEYTAB,
     SERVER_PRINCIPAL,
@@ -456,6 +458,19 @@ object KyuubiConf {
     .transform(_.toLowerCase(Locale.ROOT))
     .createWithDefault(SaslQOP.AUTH.toString)
 
+  val FRONTEND_REST_BIND_HOST: OptionalConfigEntry[String] = buildConf("frontend.rest.bind.host")
+    .doc("Hostname or IP of the machine on which to run the REST frontend service.")
+    .version("1.4.0")
+    .stringConf
+    .createOptional
+
+  val FRONTEND_REST_BIND_PORT: ConfigEntry[Int] = buildConf("frontend.rest.bind.port")
+    .doc("Port of the machine on which to run the REST frontend service.")
+    .version("1.4.0")
+    .intConf
+    .checkValue(p => p == 0 || (p > 1024 && p < 65535), "Invalid Port number")
+    .createWithDefault(10099)
+
   /////////////////////////////////////////////////////////////////////////////////////////////////
   //                                 SQL Engine Configuration                                    //
   /////////////////////////////////////////////////////////////////////////////////////////////////
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/NoopServer.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/NoopServer.scala
index 644c8ba..ebaa32e 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/NoopServer.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/NoopServer.scala
@@ -25,7 +25,8 @@ class NoopServer extends Serverable("noop") {
   private val OOMHook = new Runnable { override def run(): Unit = stop() }
 
   override val backendService = new NoopBackendService
-  val frontendService = new ThriftFrontendService(backendService, OOMHook)
+  protected val frontendService: AbstractFrontendService =
+    new ThriftFrontendService(backendService, OOMHook)
 
   override def initialize(conf: KyuubiConf): Unit = {
     addService(frontendService)
diff --git a/kyuubi-server/pom.xml b/kyuubi-server/pom.xml
index 13820a4..2bb821a 100644
--- a/kyuubi-server/pom.xml
+++ b/kyuubi-server/pom.xml
@@ -67,6 +67,21 @@
         </dependency>
 
         <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-server</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.glassfish.jersey.containers</groupId>
+            <artifactId>jersey-container-servlet-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.glassfish.jersey.inject</groupId>
+            <artifactId>jersey-hk2</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.kyuubi</groupId>
             <artifactId>kyuubi-common_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/RestFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/RestFrontendService.scala
new file mode 100644
index 0000000..b84c34a
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/RestFrontendService.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.kyuubi.server
+
+import java.net.InetAddress
+
+import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector}
+import org.eclipse.jetty.server.handler.ErrorHandler
+import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler
+
+import org.apache.kyuubi.{KyuubiException, Logging, Utils}
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.config.KyuubiConf.{FRONTEND_REST_BIND_HOST, FRONTEND_REST_BIND_PORT}
+import org.apache.kyuubi.server.api.ApiUtils
+import org.apache.kyuubi.service.{AbstractFrontendService, BackendService, ServiceState}
+
+/**
+ * A frontend service based on RESTful api via HTTP protocol.
+ * Note: Currently, it only be used in the Kyuubi Server side.
+ */
+private[server] class RestFrontendService private(name: String, be: BackendService)
+  extends AbstractFrontendService(name, be) with Logging {
+
+  def this(be: BackendService) = {
+    this(classOf[RestFrontendService].getSimpleName, be)
+  }
+
+  var serverAddr: InetAddress = _
+  var portNum: Int = _
+  var jettyServer: Server = _
+  var connector: ServerConnector = _
+
+  @volatile protected var isStarted = false
+
+  override def initialize(conf: KyuubiConf): Unit = synchronized {
+    val serverHost = conf.get(FRONTEND_REST_BIND_HOST)
+    serverAddr = serverHost.map(InetAddress.getByName).getOrElse(Utils.findLocalInetAddress)
+    portNum = conf.get(FRONTEND_REST_BIND_PORT)
+
+    jettyServer = new Server()
+
+    // set error handler
+    val errorHandler = new ErrorHandler()
+    errorHandler.setShowStacks(true)
+    errorHandler.setServer(jettyServer)
+    jettyServer.addBean(errorHandler)
+
+    jettyServer.setHandler(ApiUtils.getServletHandler(be))
+
+    connector = new ServerConnector(
+      jettyServer,
+      null,
+      new ScheduledExecutorScheduler(s"${this.name}-JettyScheduler", true),
+      null,
+      -1,
+      -1,
+      Array(new HttpConnectionFactory(new HttpConfiguration())): _*)
+    connector.setPort(portNum)
+    connector.setHost(serverAddr.getCanonicalHostName)
+    connector.setReuseAddress(!Utils.isWindows)
+    connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8))
+
+    super.initialize(conf)
+  }
+
+  override def connectionUrl(server: Boolean = false): String = {
+    getServiceState match {
+      case s @ ServiceState.LATENT => throw new IllegalStateException(s"Illegal Service State: $s")
+      case _ =>
+        s"${serverAddr.getCanonicalHostName}:$portNum"
+    }
+  }
+
+  override def start(): Unit = {
+    if (!isStarted) {
+      try {
+        connector.start()
+        jettyServer.start()
+        info(s"Rest frontend service jetty server has started at ${jettyServer.getURI}.")
+      } catch {
+        case rethrow: Exception =>
+          stopHttpServer()
+          throw new KyuubiException("Cannot start rest frontend service jetty server", rethrow)
+      }
+      isStarted = true
+    }
+
+    super.start()
+  }
+
+  override def stop(): Unit = {
+    if (isStarted) {
+      stopHttpServer()
+      isStarted = false
+    }
+    super.stop()
+  }
+
+  private def stopHttpServer(): Unit = {
+    if (jettyServer != null) {
+      try {
+        jettyServer.stop()
+        info("Rest frontend service jetty server has stopped.")
+      } catch {
+        case err: Exception => error("Cannot safely stop rest frontend service jetty server", err)
+      } finally {
+        jettyServer = null
+      }
+    }
+  }
+
+}
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/api.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/api.scala
new file mode 100644
index 0000000..9313fe2
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/api.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.kyuubi.server.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.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
+import org.glassfish.jersey.server.ServerProperties
+import org.glassfish.jersey.servlet.ServletContainer
+
+import org.apache.kyuubi.service.BackendService
+
+private[api] trait ApiRequestContext {
+
+  @Context
+  protected var servletContext: ServletContext = _
+
+  @Context
+  protected var httpRequest: HttpServletRequest = _
+
+  def backendService: BackendService = BackendServiceProvider.getBackendService(servletContext)
+
+}
+
+private[api] object BackendServiceProvider {
+
+  private val attribute = getClass.getCanonicalName
+
+  def setBackendService(contextHandler: ContextHandler, be: BackendService): Unit = {
+    contextHandler.setAttribute(attribute, be)
+  }
+
+  def getBackendService(context: ServletContext): BackendService = {
+    context.getAttribute(attribute).asInstanceOf[BackendService]
+  }
+}
+
+private[server] object ApiUtils {
+
+  def getServletHandler(backendService: BackendService): ServletContextHandler = {
+    val servlet = new ServletHolder(classOf[ServletContainer])
+    servlet.setInitParameter(
+      ServerProperties.PROVIDER_PACKAGES,
+      "org.apache.kyuubi.server.api.v1")
+    servlet.setInitParameter(
+      ServerProperties.PROVIDER_CLASSNAMES,
+      "org.glassfish.jersey.jackson.JacksonFeature")
+    val handler = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
+    BackendServiceProvider.setBackendService(handler, backendService)
+    handler.setContextPath("/api")
+    handler.addServlet(servlet, "/*")
+    handler
+  }
+
+}
+
+
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala
new file mode 100644
index 0000000..e915026
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.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.kyuubi.server.api.v1
+
+import javax.ws.rs.{GET, Path, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.kyuubi.server.api.ApiRequestContext
+
+@Path("/v1")
+private[v1] class ApiRootResource extends ApiRequestContext {
+
+  @GET
+  @Path("ping")
+  @Produces(Array(MediaType.TEXT_PLAIN))
+  def ping(): String = "pong"
+
+}
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/RestFrontendServiceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/RestFrontendServiceSuite.scala
new file mode 100644
index 0000000..f453a2e
--- /dev/null
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/RestFrontendServiceSuite.scala
@@ -0,0 +1,89 @@
+/*
+ * 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.kyuubi.server
+
+import java.util.Locale
+
+import org.scalatest.time.SpanSugar._
+import scala.io.Source
+
+import org.apache.kyuubi.KyuubiFunSuite
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.service.NoopServer
+import org.apache.kyuubi.service.ServiceState._
+
+class RestFrontendServiceSuite extends KyuubiFunSuite{
+
+  test("kyuubi rest frontend service basic") {
+    val server = new RestNoopServer()
+    server.stop()
+    val conf = KyuubiConf()
+    assert(server.getServices.isEmpty)
+    assert(server.getServiceState === LATENT)
+    val e = intercept[IllegalStateException](server.connectionUrl)
+    assert(e.getMessage === "Illegal Service State: LATENT")
+    assert(server.getConf === null)
+
+    server.initialize(conf)
+    assert(server.getServiceState === INITIALIZED)
+    val frontendService = server.getServices(0).asInstanceOf[RestFrontendService]
+    assert(frontendService.getServiceState == INITIALIZED)
+    assert(server.connectionUrl.split(":").length === 2)
+    assert(server.getConf === conf)
+    assert(server.getStartTime === 0)
+    server.stop()
+
+    server.start()
+    assert(server.getServiceState === STARTED)
+    assert(frontendService.getServiceState == STARTED)
+    assert(server.getStartTime !== 0)
+    logger.info(frontendService.connectionUrl(false))
+
+    server.stop()
+    assert(server.getServiceState === STOPPED)
+    assert(frontendService.getServiceState == STOPPED)
+    server.stop()
+  }
+
+  test("kyuubi rest frontend service http basic") {
+    val server = new RestNoopServer()
+    server.stop()
+    val conf = KyuubiConf()
+    conf.set(KyuubiConf.FRONTEND_REST_BIND_HOST, "localhost")
+
+    server.initialize(conf)
+    val frontendService = server.getServices(0).asInstanceOf[RestFrontendService]
+    server.start()
+    assert(server.getServiceState === STARTED)
+    assert(frontendService.getServiceState == STARTED)
+
+    eventually(timeout(10.seconds), interval(50.milliseconds)) {
+      val html = Source.fromURL("http://localhost:10099/api/v1/ping").mkString
+      assert(html.toLowerCase(Locale.ROOT).equals("pong"))
+    }
+
+    server.stop()
+  }
+
+  class RestNoopServer extends NoopServer {
+
+    override val frontendService = new RestFrontendService(backendService)
+
+  }
+
+}
diff --git a/pom.xml b/pom.xml
index e9d0eed..1ca49d9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
         <jakarta.servlet-api.version>4.0.4</jakarta.servlet-api.version>
         <jaxb.version>2.2.11</jaxb.version>
         <javax-activation.version>1.1.1</javax-activation.version>
+        <jersey.version>2.29</jersey.version>
         <jetty.version>9.4.41.v20210516</jetty.version>
         <kubernetes-client.version>5.5.0</kubernetes-client.version>
         <ldapsdk.version>5.1.4</ldapsdk.version>
@@ -1046,6 +1047,31 @@
                 <artifactId>scopt_${scala.binary.version}</artifactId>
                 <version>${scopt.version}</version>
             </dependency>
+
+            <!-- RESTful service required -->
+            <dependency>
+                <groupId>org.glassfish.jersey.core</groupId>
+                <artifactId>jersey-server</artifactId>
+                <version>${jersey.version}</version>
+                <exclusions>
+                    <exclusion>
+                        <groupId>jakarta.xml.bind</groupId>
+                        <artifactId>jakarta.xml.bind-api</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+
+            <dependency>
+                <groupId>org.glassfish.jersey.containers</groupId>
+                <artifactId>jersey-container-servlet-core</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.glassfish.jersey.inject</groupId>
+                <artifactId>jersey-hk2</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>